gaborkaszab commented on code in PR #14166:
URL: https://github.com/apache/iceberg/pull/14166#discussion_r2763955445


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -1239,7 +1257,8 @@ protected RESTTableOperations newTableOps(
         updateType,
         createChanges,
         current,
-        supportedEndpoints);
+        supportedEndpoints,
+        ImmutableMap.of());

Review Comment:
   This could call one of the existing constructors IMO. No need to add an 
empty map as param, RESTTableOperations can do that when calling different 
versions of the constructors internally.



##########
core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java:
##########
@@ -61,6 +62,7 @@ enum UpdateType {
   private final List<MetadataUpdate> createChanges;
   private final TableMetadata replaceBase;
   private final Set<Endpoint> endpoints;
+  private final Map<String, String> queryParams;

Review Comment:
   For now we only pass snapshot mode. I'm leaning towards only having that as 
a member and not a general queryParams, to have better understanding what this 
is.



##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1024,6 +1024,81 @@ public void testTableCredential(String oauth2ServerUri) {
         oauth2ServerUri);
   }
 
+  @Test
+  public void testTableRefreshLoadsRefsOnly() {
+    RESTCatalogAdapter adapter = Mockito.spy(new 
RESTCatalogAdapter(backendCatalog));
+
+    RESTCatalog catalog =
+        new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) 
-> adapter);
+    catalog.initialize(
+        "test",
+        ImmutableMap.of(
+            CatalogProperties.URI,
+            "ignored",
+            CatalogProperties.FILE_IO_IMPL,
+            "org.apache.iceberg.inmemory.InMemoryFileIO",
+            // default loading to refs only
+            RESTCatalogProperties.SNAPSHOT_LOADING_MODE,
+            SnapshotMode.REFS.name()));
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    // Create a table with multiple snapshots
+    Table table = catalog.createTable(TABLE, SCHEMA);
+    table

Review Comment:
   I know this is how other relevant tests set up such a table, but I think 
there is a way shorter way:
   ```
   catalog.createNamespace(TABLE.namespace());
   Table table = catalog.createTable(TABLE, SCHEMA);
   table.newAppend().appendFile(FILE_A).commit();
   table.newAppend().appendFile(FILE_B).commit();
   ```



##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -3412,7 +3489,8 @@ protected RESTTableOperations newTableOps(
           Supplier<Map<String, String>> mutationHeaders,
           FileIO fileIO,
           TableMetadata current,
-          Set<Endpoint> supportedEndpoints) {
+          Set<Endpoint> supportedEndpoints,
+          Map<String, String> queryParams) {

Review Comment:
   I know it's not the best design, but other projects might override 
`newTableOps` to inject their own way of creating RESTTableOps. If you change 
the signature, they probably have to change their code too. As written in my 
earlier comment, we can avoid changing this by directly using `snapshotMode` to 
the ops constructor.



##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -3393,7 +3469,8 @@ class CustomTableOps extends RESTTableOperations {
           FileIO io,
           TableMetadata current,
           Set<Endpoint> endpoints) {
-        super(client, path, readHeaders, mutationHeaders, io, current, 
endpoints);
+        super(
+            client, path, readHeaders, mutationHeaders, io, current, 
endpoints, ImmutableMap.of());

Review Comment:
   I don't think this is necessary



##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -569,7 +575,8 @@ private Supplier<BaseTable> createTableSupplier(
       SessionContext context,
       RESTClient tableClient,
       Map<String, String> tableConf,
-      List<Credential> credentials) {
+      List<Credential> credentials,
+      Map<String, String> queryParams) {

Review Comment:
   Since snapshotMode is a member of the class, I wouldn't extend this and the 
newTableOps functions with new params, I'd use the member directly where we 
create the ops object.



##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1024,6 +1024,81 @@ public void testTableCredential(String oauth2ServerUri) {
         oauth2ServerUri);
   }
 
+  @Test
+  public void testTableRefreshLoadsRefsOnly() {
+    RESTCatalogAdapter adapter = Mockito.spy(new 
RESTCatalogAdapter(backendCatalog));
+
+    RESTCatalog catalog =
+        new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) 
-> adapter);
+    catalog.initialize(
+        "test",
+        ImmutableMap.of(
+            CatalogProperties.URI,
+            "ignored",
+            CatalogProperties.FILE_IO_IMPL,
+            "org.apache.iceberg.inmemory.InMemoryFileIO",
+            // default loading to refs only
+            RESTCatalogProperties.SNAPSHOT_LOADING_MODE,
+            SnapshotMode.REFS.name()));
+
+    if (requiresNamespaceCreate()) {

Review Comment:
   This is always tru in this test suite. no need to check



##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1024,6 +1024,81 @@ public void testTableCredential(String oauth2ServerUri) {
         oauth2ServerUri);
   }
 
+  @Test
+  public void testTableRefreshLoadsRefsOnly() {

Review Comment:
   nit: no need for the 'test' prefix in the name.



##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1024,6 +1024,81 @@ public void testTableCredential(String oauth2ServerUri) {
         oauth2ServerUri);
   }
 
+  @Test
+  public void testTableRefreshLoadsRefsOnly() {
+    RESTCatalogAdapter adapter = Mockito.spy(new 
RESTCatalogAdapter(backendCatalog));
+
+    RESTCatalog catalog =
+        new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) 
-> adapter);
+    catalog.initialize(
+        "test",
+        ImmutableMap.of(
+            CatalogProperties.URI,
+            "ignored",
+            CatalogProperties.FILE_IO_IMPL,
+            "org.apache.iceberg.inmemory.InMemoryFileIO",
+            // default loading to refs only
+            RESTCatalogProperties.SNAPSHOT_LOADING_MODE,
+            SnapshotMode.REFS.name()));
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    // Create a table with multiple snapshots
+    Table table = catalog.createTable(TABLE, SCHEMA);
+    table
+        .newFastAppend()
+        .appendFile(
+            DataFiles.builder(PartitionSpec.unpartitioned())
+                .withPath("/path/to/data-a.parquet")
+                .withFileSizeInBytes(10)
+                .withRecordCount(2)
+                .build())
+        .commit();
+
+    table
+        .newFastAppend()
+        .appendFile(
+            DataFiles.builder(PartitionSpec.unpartitioned())
+                .withPath("/path/to/data-b.parquet")
+                .withFileSizeInBytes(10)
+                .withRecordCount(2)
+                .build())
+        .commit();
+
+    ResourcePaths paths = 
ResourcePaths.forCatalogProperties(Maps.newHashMap());
+
+    Table refsTable = catalog.loadTable(TABLE);
+    refsTable.refresh();
+
+    // don't call snapshots() directly as that would cause to load all 
snapshots. Instead,
+    // make sure the snapshots field holds exactly 1 snapshot
+    assertThat(((BaseTable) refsTable).operations().current())
+        .extracting("snapshots")
+        .asInstanceOf(InstanceOfAssertFactories.list(Snapshot.class))
+        .hasSize(1);
+
+    assertThat(refsTable.currentSnapshot()).isEqualTo(table.currentSnapshot());
+
+    // verify that the table was loaded with the refs argument
+    verify(adapter, times(2))
+        .execute(
+            reqMatcher(HTTPMethod.GET, paths.table(TABLE), Map.of(), 
Map.of("snapshots", "refs")),

Review Comment:
   `reqMatcher` was moved to another class. `matches` is how to use it now



##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1024,6 +1024,81 @@ public void testTableCredential(String oauth2ServerUri) {
         oauth2ServerUri);
   }
 
+  @Test
+  public void testTableRefreshLoadsRefsOnly() {
+    RESTCatalogAdapter adapter = Mockito.spy(new 
RESTCatalogAdapter(backendCatalog));
+
+    RESTCatalog catalog =
+        new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) 
-> adapter);
+    catalog.initialize(
+        "test",
+        ImmutableMap.of(
+            CatalogProperties.URI,
+            "ignored",
+            CatalogProperties.FILE_IO_IMPL,
+            "org.apache.iceberg.inmemory.InMemoryFileIO",
+            // default loading to refs only
+            RESTCatalogProperties.SNAPSHOT_LOADING_MODE,
+            SnapshotMode.REFS.name()));
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    // Create a table with multiple snapshots
+    Table table = catalog.createTable(TABLE, SCHEMA);
+    table
+        .newFastAppend()
+        .appendFile(
+            DataFiles.builder(PartitionSpec.unpartitioned())
+                .withPath("/path/to/data-a.parquet")
+                .withFileSizeInBytes(10)
+                .withRecordCount(2)
+                .build())
+        .commit();
+
+    table
+        .newFastAppend()
+        .appendFile(
+            DataFiles.builder(PartitionSpec.unpartitioned())
+                .withPath("/path/to/data-b.parquet")
+                .withFileSizeInBytes(10)
+                .withRecordCount(2)
+                .build())
+        .commit();
+
+    ResourcePaths paths = 
ResourcePaths.forCatalogProperties(Maps.newHashMap());
+
+    Table refsTable = catalog.loadTable(TABLE);
+    refsTable.refresh();
+
+    // don't call snapshots() directly as that would cause to load all 
snapshots. Instead,
+    // make sure the snapshots field holds exactly 1 snapshot
+    assertThat(((BaseTable) refsTable).operations().current())
+        .extracting("snapshots")
+        .asInstanceOf(InstanceOfAssertFactories.list(Snapshot.class))
+        .hasSize(1);
+
+    assertThat(refsTable.currentSnapshot()).isEqualTo(table.currentSnapshot());
+
+    // verify that the table was loaded with the refs argument
+    verify(adapter, times(2))
+        .execute(
+            reqMatcher(HTTPMethod.GET, paths.table(TABLE), Map.of(), 
Map.of("snapshots", "refs")),
+            eq(LoadTableResponse.class),
+            any(),
+            any());
+
+    // verify that all snapshots are loaded when referenced

Review Comment:
   Not entirely sure it has any added value to check this. Could be irrelevant 
for this change. What are we validating here?



##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -947,7 +956,8 @@ public Table create() {
               mutationHeaders,
               tableFileIO(context, tableConf, response.credentials()),
               response.tableMetadata(),
-              endpoints);
+              endpoints,
+              ImmutableMap.of());

Review Comment:
   no need for this change, calling the existing constructor would do the same 
for us,



##########
core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java:
##########
@@ -1024,6 +1024,81 @@ public void testTableCredential(String oauth2ServerUri) {
         oauth2ServerUri);
   }
 
+  @Test
+  public void testTableRefreshLoadsRefsOnly() {
+    RESTCatalogAdapter adapter = Mockito.spy(new 
RESTCatalogAdapter(backendCatalog));
+
+    RESTCatalog catalog =
+        new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) 
-> adapter);
+    catalog.initialize(
+        "test",
+        ImmutableMap.of(
+            CatalogProperties.URI,
+            "ignored",
+            CatalogProperties.FILE_IO_IMPL,
+            "org.apache.iceberg.inmemory.InMemoryFileIO",
+            // default loading to refs only
+            RESTCatalogProperties.SNAPSHOT_LOADING_MODE,
+            SnapshotMode.REFS.name()));
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    // Create a table with multiple snapshots
+    Table table = catalog.createTable(TABLE, SCHEMA);
+    table
+        .newFastAppend()
+        .appendFile(
+            DataFiles.builder(PartitionSpec.unpartitioned())
+                .withPath("/path/to/data-a.parquet")
+                .withFileSizeInBytes(10)
+                .withRecordCount(2)
+                .build())
+        .commit();
+
+    table
+        .newFastAppend()
+        .appendFile(
+            DataFiles.builder(PartitionSpec.unpartitioned())
+                .withPath("/path/to/data-b.parquet")
+                .withFileSizeInBytes(10)
+                .withRecordCount(2)
+                .build())
+        .commit();
+
+    ResourcePaths paths = 
ResourcePaths.forCatalogProperties(Maps.newHashMap());

Review Comment:
   Is there a reason why `RESOURCE_PATHS` is not sufficient for this test?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to