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 2022/09/19 12:28:20 UTC

[GitHub] [iceberg] findepi opened a new pull request, #5795: Retain table statistics during orphan files removal

findepi opened a new pull request, #5795:
URL: https://github.com/apache/iceberg/pull/5795

   Do not delete table statistics files when running remove_orphan_files.
   


-- 
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: issues-unsubscribe@iceberg.apache.org

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] findepi commented on pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#issuecomment-1261976192

   > This looks fine, but it requires the public API to return statisticsFiles from Table so we should get that one in first.
   
   please see my response: https://github.com/apache/iceberg/pull/5795#discussion_r983265802


-- 
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: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#discussion_r982951584


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -876,6 +890,82 @@ protected long waitUntilAfter(long timestampMillis) {
     return current;
   }
 
+  @Test
+  public void testRemoveOrphanFilesWithStatisticFiles() throws Exception {
+    Table table =
+        TABLES.create(
+            SCHEMA,
+            PartitionSpec.unpartitioned(),
+            ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"),
+            tableLocation);
+
+    List<ThreeColumnRecord> records =
+        Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA"));
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+    df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation);
+
+    table.refresh();
+    long snapshotId = table.currentSnapshot().snapshotId();
+    long snapshotSequenceNumber = table.currentSnapshot().sequenceNumber();
+
+    File statsLocation =
+        new File(new URI(tableLocation))
+            .toPath()
+            .resolve("data")
+            .resolve("some-stats-file")
+            .toFile();
+    StatisticsFile statisticsFile;
+    try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) {
+      puffinWriter.add(
+          new Blob(
+              "some-blob-type",
+              ImmutableList.of(1),
+              snapshotId,
+              snapshotSequenceNumber,
+              ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8))));
+      puffinWriter.finish();
+      statisticsFile =
+          new GenericStatisticsFile(
+              snapshotId,
+              statsLocation.toString(),
+              puffinWriter.fileSize(),
+              puffinWriter.footerSize(),
+              puffinWriter.writtenBlobsMetadata().stream()
+                  .map(GenericBlobMetadata::from)
+                  .collect(ImmutableList.toImmutableList()));
+    }
+
+    Transaction transaction = table.newTransaction();
+    transaction.updateStatistics().setStatistics(snapshotId, statisticsFile).commit();
+    transaction.commitTransaction();
+
+    SparkActions.get()
+        .deleteOrphanFiles(table)
+        .olderThan(System.currentTimeMillis() + TimeUnit.DAYS.toMillis(365))
+        .execute();
+
+    Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue();
+    Assertions.assertThat(statsLocation.length())
+        .as("stats file length")
+        .isEqualTo(statisticsFile.fileSizeInBytes());
+
+    transaction = table.newTransaction();
+    transaction.updateStatistics().removeStatistics(statisticsFile.snapshotId()).commit();
+    transaction.commitTransaction();
+
+    DeleteOrphanFiles.Result result =
+        SparkActions.get()
+            .deleteOrphanFiles(table)
+            .olderThan(System.currentTimeMillis() + TimeUnit.DAYS.toMillis(365))

Review Comment:
   Other tests use `+ 1000` rather than 365 days. Why use 1 year in the future?



-- 
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: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#discussion_r982951929


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -876,6 +890,82 @@ protected long waitUntilAfter(long timestampMillis) {
     return current;
   }
 
+  @Test
+  public void testRemoveOrphanFilesWithStatisticFiles() throws Exception {

Review Comment:
   Overall the test 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 #5795: Retain table statistics during orphan files removal

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

   Thanks, @findepi! I merged this.


-- 
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: issues-unsubscribe@iceberg.apache.org

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] findepi commented on a diff in pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#discussion_r983265802


##########
core/src/main/java/org/apache/iceberg/ReachableFileUtil.java:
##########
@@ -113,4 +113,21 @@ public static List<String> manifestListLocations(Table table) {
     }
     return manifestListLocations;
   }
+
+  /**
+   * Returns locations of statistics files in a table.
+   *
+   * @param table table for which statistics files needs to be listed
+   * @return the location of statistics files
+   */
+  public static List<String> statisticsFilesLocations(Table table) {
+    List<String> statisticsFilesLocations = Lists.newArrayList();
+    TableOperations ops = ((HasTableOperations) table).operations();

Review Comment:
   @rdblue here I am following existing code in the `ReachableFileUtil` class 
   
   https://github.com/apache/iceberg/blob/c07f2aabc0a1d02f068ecf1514d2479c0fbdd3b0/core/src/main/java/org/apache/iceberg/ReachableFileUtil.java#L62-L64
   
   since this class already casts to `HasTableOperations`, I assumed it's OK to do that and therefore removed public API changes from this PR.
   If you want first merge Table API changes, the https://github.com/apache/iceberg/pull/4741 is ready for your review



-- 
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: issues-unsubscribe@iceberg.apache.org

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 #5795: Retain table statistics during orphan files removal

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


-- 
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: issues-unsubscribe@iceberg.apache.org

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] findepi commented on pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#issuecomment-1259378754

   Rebased after https://github.com/apache/iceberg/pull/5794 is merged.
   
   @rdblue please take a look


-- 
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: issues-unsubscribe@iceberg.apache.org

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] findepi commented on a diff in pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#discussion_r983285734


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -876,6 +890,82 @@ protected long waitUntilAfter(long timestampMillis) {
     return current;
   }
 
+  @Test
+  public void testRemoveOrphanFilesWithStatisticFiles() throws Exception {
+    Table table =
+        TABLES.create(
+            SCHEMA,
+            PartitionSpec.unpartitioned(),
+            ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"),
+            tableLocation);
+
+    List<ThreeColumnRecord> records =
+        Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA"));
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+    df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation);
+
+    table.refresh();
+    long snapshotId = table.currentSnapshot().snapshotId();
+    long snapshotSequenceNumber = table.currentSnapshot().sequenceNumber();
+
+    File statsLocation =
+        new File(new URI(tableLocation))
+            .toPath()
+            .resolve("data")
+            .resolve("some-stats-file")
+            .toFile();
+    StatisticsFile statisticsFile;
+    try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) {
+      puffinWriter.add(
+          new Blob(
+              "some-blob-type",
+              ImmutableList.of(1),
+              snapshotId,
+              snapshotSequenceNumber,
+              ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8))));
+      puffinWriter.finish();
+      statisticsFile =
+          new GenericStatisticsFile(

Review Comment:
   it currently does not have the information for `org.apache.iceberg.StatisticsFile#snapshotId` field.



-- 
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: issues-unsubscribe@iceberg.apache.org

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] findepi commented on pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#issuecomment-1263864099

   thank you @rdblue for the merge!


-- 
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: issues-unsubscribe@iceberg.apache.org

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] findepi commented on a diff in pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#discussion_r983266239


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -876,6 +890,82 @@ protected long waitUntilAfter(long timestampMillis) {
     return current;
   }
 
+  @Test
+  public void testRemoveOrphanFilesWithStatisticFiles() throws Exception {

Review Comment:
   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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] findepi commented on pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#issuecomment-1262030928

   Applied or responded to comments.
   I didn't do anything with the public Table API yet.
   I think, however, it shouldn't be a blocker, since the affected `ReachableFileUtil` class already depends on non-API information in other methods. We can improve `ReachableFileUtil` after https://github.com/apache/iceberg/pull/4741, or we can land https://github.com/apache/iceberg/pull/4741 first and improve here.
   
   @rdblue please take another look.


-- 
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: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#discussion_r982951076


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -876,6 +890,82 @@ protected long waitUntilAfter(long timestampMillis) {
     return current;
   }
 
+  @Test
+  public void testRemoveOrphanFilesWithStatisticFiles() throws Exception {
+    Table table =
+        TABLES.create(
+            SCHEMA,
+            PartitionSpec.unpartitioned(),
+            ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"),
+            tableLocation);
+
+    List<ThreeColumnRecord> records =
+        Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA"));
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+    df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation);
+
+    table.refresh();
+    long snapshotId = table.currentSnapshot().snapshotId();
+    long snapshotSequenceNumber = table.currentSnapshot().sequenceNumber();
+
+    File statsLocation =
+        new File(new URI(tableLocation))
+            .toPath()
+            .resolve("data")
+            .resolve("some-stats-file")
+            .toFile();
+    StatisticsFile statisticsFile;
+    try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) {
+      puffinWriter.add(
+          new Blob(
+              "some-blob-type",
+              ImmutableList.of(1),
+              snapshotId,
+              snapshotSequenceNumber,
+              ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8))));
+      puffinWriter.finish();
+      statisticsFile =
+          new GenericStatisticsFile(

Review Comment:
   Can `PuffinWriter` expose `toStatisticsFile` instead of relying on the caller to do this work?



-- 
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: issues-unsubscribe@iceberg.apache.org

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] findepi commented on a diff in pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#discussion_r983278571


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -876,6 +890,82 @@ protected long waitUntilAfter(long timestampMillis) {
     return current;
   }
 
+  @Test
+  public void testRemoveOrphanFilesWithStatisticFiles() throws Exception {
+    Table table =
+        TABLES.create(
+            SCHEMA,
+            PartitionSpec.unpartitioned(),
+            ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"),
+            tableLocation);
+
+    List<ThreeColumnRecord> records =
+        Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA"));
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+    df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation);
+
+    table.refresh();
+    long snapshotId = table.currentSnapshot().snapshotId();
+    long snapshotSequenceNumber = table.currentSnapshot().sequenceNumber();
+
+    File statsLocation =
+        new File(new URI(tableLocation))
+            .toPath()
+            .resolve("data")
+            .resolve("some-stats-file")
+            .toFile();
+    StatisticsFile statisticsFile;
+    try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) {
+      puffinWriter.add(
+          new Blob(
+              "some-blob-type",
+              ImmutableList.of(1),
+              snapshotId,
+              snapshotSequenceNumber,
+              ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8))));
+      puffinWriter.finish();
+      statisticsFile =
+          new GenericStatisticsFile(
+              snapshotId,
+              statsLocation.toString(),
+              puffinWriter.fileSize(),
+              puffinWriter.footerSize(),
+              puffinWriter.writtenBlobsMetadata().stream()
+                  .map(GenericBlobMetadata::from)
+                  .collect(ImmutableList.toImmutableList()));
+    }
+
+    Transaction transaction = table.newTransaction();
+    transaction.updateStatistics().setStatistics(snapshotId, statisticsFile).commit();
+    transaction.commitTransaction();
+
+    SparkActions.get()
+        .deleteOrphanFiles(table)
+        .olderThan(System.currentTimeMillis() + TimeUnit.DAYS.toMillis(365))
+        .execute();
+
+    Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue();
+    Assertions.assertThat(statsLocation.length())
+        .as("stats file length")
+        .isEqualTo(statisticsFile.fileSizeInBytes());
+
+    transaction = table.newTransaction();
+    transaction.updateStatistics().removeStatistics(statisticsFile.snapshotId()).commit();
+    transaction.commitTransaction();
+
+    DeleteOrphanFiles.Result result =
+        SparkActions.get()
+            .deleteOrphanFiles(table)
+            .olderThan(System.currentTimeMillis() + TimeUnit.DAYS.toMillis(365))

Review Comment:
   I saw tests using
   ```java
   Thread.sleep(1000);
   ...
   .deleteOrphanFiles(table)
   .olderThan(System.currentTimeMillis())
   ```
   
   i wanted to avoid sleep in the test, so that it run faster on CI, that's why i didn't follow existing pattern that i found.
   
   i will change to `+ 1000`
   



-- 
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: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#discussion_r982952940


##########
core/src/main/java/org/apache/iceberg/ReachableFileUtil.java:
##########
@@ -113,4 +113,21 @@ public static List<String> manifestListLocations(Table table) {
     }
     return manifestListLocations;
   }
+
+  /**
+   * Returns locations of statistics files in a table.
+   *
+   * @param table table for which statistics files needs to be listed
+   * @return the location of statistics files
+   */
+  public static List<String> statisticsFilesLocations(Table table) {
+    List<String> statisticsFilesLocations = Lists.newArrayList();
+    TableOperations ops = ((HasTableOperations) table).operations();

Review Comment:
   Looks like this depends on the public API. Let's get that one in first so we don't need to go around the API 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] findepi commented on pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#issuecomment-1263357400

   > This looks fine, but it requires the public API to return statisticsFiles from Table so we should get that one in first.
   
   Done now.
   
   @rdblue please take another look


-- 
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: issues-unsubscribe@iceberg.apache.org

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] findepi commented on pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#issuecomment-1263355996

   (just rebased after https://github.com/apache/iceberg/pull/4741 merged, no changes yet)


-- 
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: issues-unsubscribe@iceberg.apache.org

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] findepi commented on pull request #5795: Retain table statistics during orphan files removal

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #5795:
URL: https://github.com/apache/iceberg/pull/5795#issuecomment-1253515066

   rebased after https://github.com/apache/iceberg/pull/5799 merged, no other changes
   
   currently, depends on https://github.com/apache/iceberg/pull/5794


-- 
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: issues-unsubscribe@iceberg.apache.org

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