You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/08/14 18:21:49 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #1344: Expire snapshots action without cache

RussellSpitzer opened a new pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344


   Requires #1342
   
   
   Removes the necessity of a Cache from ExpireSnapshotsAction. Instead uses a Static view of the TableMetadata enabled by #1342 to preserve a listing of all metadata/data files from a previous point in time.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#discussion_r473360622



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -86,17 +90,22 @@ protected String metadataTableName(MetadataTableType type) {
   }
 
   protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
-    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return buildValidDataFileDF(spark, table().toString());
+  }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark, String tableName) {
+    String allDataFilesMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_DATA_FILES);
     return spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
   }
 
-  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
-    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark, String tableName) {
+    String allManifestsMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_MANIFESTS);
     return spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path as file_path");
   }
 
-  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
-    List<String> manifestLists = getManifestListPaths(table);
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, String tableName, TableOperations ops) {
+    Table snapshot = new BaseTable(ops, tableName);
+    List<String> manifestLists = getManifestListPaths(snapshot);

Review comment:
       Sounds good to me




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#issuecomment-677767209


   Merging. Looks good.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#issuecomment-676733881


   This looks great, @RussellSpitzer! I had a few minor comments about structure, but overall it has no major issues.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#discussion_r473360899



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -86,17 +90,22 @@ protected String metadataTableName(MetadataTableType type) {
   }
 
   protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
-    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return buildValidDataFileDF(spark, table().toString());
+  }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark, String tableName) {
+    String allDataFilesMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_DATA_FILES);
     return spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
   }
 
-  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
-    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark, String tableName) {
+    String allManifestsMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_MANIFESTS);
     return spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path as file_path");
   }
 
-  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
-    List<String> manifestLists = getManifestListPaths(table);
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, String tableName, TableOperations ops) {
+    Table snapshot = new BaseTable(ops, tableName);

Review comment:
       True, I'll just switch it to table
   Table table = Table




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344


   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#issuecomment-676616324


   @aokolnychyi + @rdblue this is the followup to the StaticTableOperation ticket #1342 using the improvement inside ExpireSnapshotActions, now with no caching!


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#discussion_r473267204



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -86,17 +90,22 @@ protected String metadataTableName(MetadataTableType type) {
   }
 
   protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
-    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return buildValidDataFileDF(spark, table().toString());
+  }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark, String tableName) {
+    String allDataFilesMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_DATA_FILES);
     return spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
   }
 
-  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
-    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark, String tableName) {

Review comment:
       We add these 2 arg versions so that we can specify metadata Json files directly, the single arg versions just use the current table state as before.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#discussion_r473355211



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -147,49 +149,41 @@ public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
 
   @Override
   public ExpireSnapshotsActionResult execute() {
-    Dataset<Row> originalFiles = null;
-    try {
-      // Metadata before Expiration
-      originalFiles = buildValidFileDF().persist();
-      // Action to trigger persist
-      originalFiles.count();
-
-      // Perform Expiration
-      ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
-      for (final Long id : expireSnapshotIdValues) {
-        expireSnaps = expireSnaps.expireSnapshotId(id);
-      }
-
-      if (expireOlderThanValue != null) {
-        expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
-      }
-
-      if (retainLastValue != null) {
-        expireSnaps = expireSnaps.retainLast(retainLastValue);
-      }
-
-      expireSnaps.commit();
-
-      // Metadata after Expiration
-      Dataset<Row> validFiles = buildValidFileDF();
-      Dataset<Row> filesToDelete = originalFiles.except(validFiles);
-
-      return deleteFiles(filesToDelete.toLocalIterator());
-    } finally {
-      if (originalFiles != null) {
-        originalFiles.unpersist();
-      }
+    // Metadata before Expiration
+    Dataset<Row>  originalFiles = buildValidFileDF(ops.current());
+
+    // Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    for (final Long id : expireSnapshotIdValues) {
+      expireSnaps = expireSnaps.expireSnapshotId(id);
+    }
+
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
     }
+
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF(ops.refresh());
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    return deleteFiles(filesToDelete.toLocalIterator());
   }
 
   private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
     return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
   }
 
-  private Dataset<Row> buildValidFileDF() {
-    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
-        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
-        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  private Dataset<Row> buildValidFileDF(TableMetadata metadata) {
+    StaticTableOperations staticOps = new StaticTableOperations(metadata.metadataFileLocation(), table.io());

Review comment:
       Minor: the metadata file location is passed to `buildManifestFileDF` and `buildValidDataFileDF`, but `StaticTableOperations` is passed into `buildManifestListDF`. I think it would make a more consistent API if the location were also passed to `buildManifestListDF`.
   
   I know that the difference is that the method accepts a `Table` and doesn't use a metadata table. But it would be a bit cleaner to pass the base `Table` and metadata location, then create the `StaticTableOperations` in that method rather than here.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#discussion_r473567264



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -86,27 +91,36 @@ protected String metadataTableName(MetadataTableType type) {
   }
 
   protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
-    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return buildValidDataFileDF(spark, table().toString());
+  }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark, String tableName) {
+    String allDataFilesMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_DATA_FILES);
     return spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
   }
 
-  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
-    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark, String tableName) {
+    String allManifestsMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_MANIFESTS);
     return spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path as file_path");
   }
 
   protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
-    List<String> manifestLists = getManifestListPaths(table);
+    List<String> manifestLists = getManifestListPaths(table.snapshots());
     return spark.createDataset(manifestLists, Encoders.STRING()).toDF("file_path");
   }
 
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, String metadataFileLocation) {

Review comment:
       You cannot pass a pure table name here since we aren't looking up the table using Spark, this path is for metadataFileLocation based tables only.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#discussion_r473351737



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -86,17 +90,22 @@ protected String metadataTableName(MetadataTableType type) {
   }
 
   protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
-    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return buildValidDataFileDF(spark, table().toString());
+  }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark, String tableName) {
+    String allDataFilesMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_DATA_FILES);
     return spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
   }
 
-  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
-    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark, String tableName) {
+    String allManifestsMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_MANIFESTS);
     return spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path as file_path");
   }
 
-  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
-    List<String> manifestLists = getManifestListPaths(table);
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, String tableName, TableOperations ops) {
+    Table snapshot = new BaseTable(ops, tableName);

Review comment:
       I think it is misleading to use `snapshot` here, since that term usually refers to a version of a table, not a table itself.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#issuecomment-677783597


   Thanks!


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#discussion_r473567025



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -147,49 +149,41 @@ public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
 
   @Override
   public ExpireSnapshotsActionResult execute() {
-    Dataset<Row> originalFiles = null;
-    try {
-      // Metadata before Expiration
-      originalFiles = buildValidFileDF().persist();
-      // Action to trigger persist
-      originalFiles.count();
-
-      // Perform Expiration
-      ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
-      for (final Long id : expireSnapshotIdValues) {
-        expireSnaps = expireSnaps.expireSnapshotId(id);
-      }
-
-      if (expireOlderThanValue != null) {
-        expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
-      }
-
-      if (retainLastValue != null) {
-        expireSnaps = expireSnaps.retainLast(retainLastValue);
-      }
-
-      expireSnaps.commit();
-
-      // Metadata after Expiration
-      Dataset<Row> validFiles = buildValidFileDF();
-      Dataset<Row> filesToDelete = originalFiles.except(validFiles);
-
-      return deleteFiles(filesToDelete.toLocalIterator());
-    } finally {
-      if (originalFiles != null) {
-        originalFiles.unpersist();
-      }
+    // Metadata before Expiration
+    Dataset<Row>  originalFiles = buildValidFileDF(ops.current());
+
+    // Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    for (final Long id : expireSnapshotIdValues) {
+      expireSnaps = expireSnaps.expireSnapshotId(id);
+    }
+
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
     }
+
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF(ops.refresh());
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    return deleteFiles(filesToDelete.toLocalIterator());
   }
 
   private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
     return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
   }
 
-  private Dataset<Row> buildValidFileDF() {
-    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
-        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
-        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  private Dataset<Row> buildValidFileDF(TableMetadata metadata) {
+    StaticTableOperations staticOps = new StaticTableOperations(metadata.metadataFileLocation(), table.io());

Review comment:
       I think I understand what you are asking for here, but I'm not sure I like how it looks since I end up with two methods, one of which takes metadataFileLocation and one which takes "Table"
   
   The metadataFileLocation version makes the StaticOps and BaseTable from them and passes to the table method,
   while the "table" method is used for the version used by Orphan files.
   
   Take a look  at the new version and see if we are on the same page




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1344: Expire snapshots action without cache

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1344:
URL: https://github.com/apache/iceberg/pull/1344#discussion_r473356757



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -86,17 +90,22 @@ protected String metadataTableName(MetadataTableType type) {
   }
 
   protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
-    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return buildValidDataFileDF(spark, table().toString());
+  }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark, String tableName) {
+    String allDataFilesMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_DATA_FILES);
     return spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
   }
 
-  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
-    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark, String tableName) {
+    String allManifestsMetadataTable = metadataTableName(tableName, MetadataTableType.ALL_MANIFESTS);
     return spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path as file_path");
   }
 
-  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
-    List<String> manifestLists = getManifestListPaths(table);
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, String tableName, TableOperations ops) {
+    Table snapshot = new BaseTable(ops, tableName);
+    List<String> manifestLists = getManifestListPaths(snapshot);

Review comment:
       What about changing `getManifestListPaths` to accept `Iterable<Snapshot>`? Then you wouldn't need to create a `BaseTable` out of a `StaticTableOperations`. Instead you could just pass `staticOps.current().snapshots()` here and `table.snapshots()` elsewhere.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org