|
23 | 23 | import static org.mockito.ArgumentMatchers.any;
|
24 | 24 | import static org.mockito.ArgumentMatchers.eq;
|
25 | 25 | import static org.mockito.Mockito.times;
|
| 26 | +import static org.mockito.Mockito.verify; |
26 | 27 |
|
27 | 28 | import com.fasterxml.jackson.core.JsonProcessingException;
|
28 | 29 | import com.fasterxml.jackson.databind.ObjectMapper;
|
29 | 30 | import java.io.File;
|
30 | 31 | import java.io.IOException;
|
31 | 32 | import java.nio.file.Path;
|
32 | 33 | import java.util.Map;
|
| 34 | +import java.util.Set; |
33 | 35 | import java.util.UUID;
|
34 | 36 | import java.util.concurrent.atomic.AtomicInteger;
|
35 | 37 | import java.util.function.Consumer;
|
| 38 | +import java.util.stream.Collectors; |
36 | 39 | import org.apache.hadoop.conf.Configuration;
|
37 | 40 | import org.apache.iceberg.AssertHelpers;
|
38 | 41 | import org.apache.iceberg.CatalogProperties;
|
39 | 42 | import org.apache.iceberg.DataFiles;
|
40 | 43 | import org.apache.iceberg.FileScanTask;
|
41 | 44 | import org.apache.iceberg.PartitionSpec;
|
42 | 45 | import org.apache.iceberg.Schema;
|
| 46 | +import org.apache.iceberg.SnapshotRef; |
43 | 47 | import org.apache.iceberg.Table;
|
| 48 | +import org.apache.iceberg.TableMetadata; |
44 | 49 | import org.apache.iceberg.catalog.CatalogTests;
|
45 | 50 | import org.apache.iceberg.catalog.SessionCatalog;
|
46 | 51 | import org.apache.iceberg.catalog.TableIdentifier;
|
|
49 | 54 | import org.apache.iceberg.metrics.MetricsReport;
|
50 | 55 | import org.apache.iceberg.metrics.MetricsReporter;
|
51 | 56 | import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
|
| 57 | +import org.apache.iceberg.relocated.com.google.common.collect.Maps; |
52 | 58 | import org.apache.iceberg.rest.RESTCatalogAdapter.HTTPMethod;
|
| 59 | +import org.apache.iceberg.rest.RESTSessionCatalog.SnapshotMode; |
53 | 60 | import org.apache.iceberg.rest.auth.AuthSessionUtil;
|
54 | 61 | import org.apache.iceberg.rest.auth.OAuth2Properties;
|
55 | 62 | import org.apache.iceberg.rest.auth.OAuth2Util;
|
@@ -737,6 +744,113 @@ public void testTableCredential() {
|
737 | 744 | ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=table-user"));
|
738 | 745 | }
|
739 | 746 |
|
| 747 | + @Test |
| 748 | + public void testSnapshotParams() { |
| 749 | + assertThat(SnapshotMode.ALL.params()).isEqualTo(ImmutableMap.of("snapshots", "all")); |
| 750 | + |
| 751 | + assertThat(SnapshotMode.REFS.params()).isEqualTo(ImmutableMap.of("snapshots", "refs")); |
| 752 | + } |
| 753 | + |
| 754 | + @Test |
| 755 | + public void testTableSnapshotLoading() { |
| 756 | + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); |
| 757 | + |
| 758 | + RESTCatalog catalog = |
| 759 | + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); |
| 760 | + catalog.initialize( |
| 761 | + "test", |
| 762 | + ImmutableMap.of( |
| 763 | + CatalogProperties.URI, |
| 764 | + "ignored", |
| 765 | + CatalogProperties.FILE_IO_IMPL, |
| 766 | + "org.apache.iceberg.io.InMemoryFileIO", |
| 767 | + // default loading to refs only |
| 768 | + "snapshot-loading-mode", |
| 769 | + "refs")); |
| 770 | + |
| 771 | + // Create a table with multiple snapshots |
| 772 | + Table table = catalog.createTable(TABLE, SCHEMA); |
| 773 | + table |
| 774 | + .newFastAppend() |
| 775 | + .appendFile( |
| 776 | + DataFiles.builder(PartitionSpec.unpartitioned()) |
| 777 | + .withPath("/path/to/data-a.parquet") |
| 778 | + .withFileSizeInBytes(10) |
| 779 | + .withRecordCount(2) |
| 780 | + .build()) |
| 781 | + .commit(); |
| 782 | + |
| 783 | + table |
| 784 | + .newFastAppend() |
| 785 | + .appendFile( |
| 786 | + DataFiles.builder(PartitionSpec.unpartitioned()) |
| 787 | + .withPath("/path/to/data-b.parquet") |
| 788 | + .withFileSizeInBytes(10) |
| 789 | + .withRecordCount(2) |
| 790 | + .build()) |
| 791 | + .commit(); |
| 792 | + |
| 793 | + ResourcePaths paths = ResourcePaths.forCatalogProperties(Maps.newHashMap()); |
| 794 | + |
| 795 | + // Respond with only referenced snapshots |
| 796 | + Answer<?> refsAnswer = |
| 797 | + invocation -> { |
| 798 | + LoadTableResponse originalResponse = (LoadTableResponse) invocation.callRealMethod(); |
| 799 | + TableMetadata fullTableMetadata = originalResponse.tableMetadata(); |
| 800 | + |
| 801 | + Set<Long> referencedSnapshotIds = |
| 802 | + fullTableMetadata.refs().values().stream() |
| 803 | + .map(SnapshotRef::snapshotId) |
| 804 | + .collect(Collectors.toSet()); |
| 805 | + |
| 806 | + TableMetadata refsMetadata = |
| 807 | + fullTableMetadata.removeSnapshotsIf( |
| 808 | + s -> !referencedSnapshotIds.contains(s.snapshotId())); |
| 809 | + |
| 810 | + return LoadTableResponse.builder() |
| 811 | + .withTableMetadata(refsMetadata) |
| 812 | + .addAllConfig(originalResponse.config()) |
| 813 | + .build(); |
| 814 | + }; |
| 815 | + |
| 816 | + Mockito.doAnswer(refsAnswer) |
| 817 | + .when(adapter) |
| 818 | + .execute( |
| 819 | + eq(HTTPMethod.GET), |
| 820 | + eq(paths.table(TABLE)), |
| 821 | + eq(ImmutableMap.of("snapshots", "refs")), |
| 822 | + any(), |
| 823 | + eq(LoadTableResponse.class), |
| 824 | + any(), |
| 825 | + any()); |
| 826 | + |
| 827 | + Table refsTables = catalog.loadTable(TABLE); |
| 828 | + |
| 829 | + assertThat(refsTables.currentSnapshot()).isEqualTo(table.currentSnapshot()); |
| 830 | + // verify that the table was loaded with the refs argument |
| 831 | + verify(adapter, times(1)) |
| 832 | + .execute( |
| 833 | + eq(HTTPMethod.GET), |
| 834 | + eq(paths.table(TABLE)), |
| 835 | + eq(ImmutableMap.of("snapshots", "refs")), |
| 836 | + any(), |
| 837 | + eq(LoadTableResponse.class), |
| 838 | + any(), |
| 839 | + any()); |
| 840 | + |
| 841 | + // verify that all snapshots are loaded when referenced |
| 842 | + assertThat(refsTables.snapshots()).containsExactlyInAnyOrderElementsOf(table.snapshots()); |
| 843 | + verify(adapter, times(1)) |
| 844 | + .execute( |
| 845 | + eq(HTTPMethod.GET), |
| 846 | + eq(paths.table(TABLE)), |
| 847 | + eq(ImmutableMap.of("snapshots", "all")), |
| 848 | + any(), |
| 849 | + eq(LoadTableResponse.class), |
| 850 | + any(), |
| 851 | + any()); |
| 852 | + } |
| 853 | + |
740 | 854 | public void testTableAuth(
|
741 | 855 | String catalogToken,
|
742 | 856 | Map<String, String> credentials,
|
|
0 commit comments