nastra commented on code in PR #13569:
URL: https://github.com/apache/iceberg/pull/13569#discussion_r2210825426


##########
flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java:
##########
@@ -77,4 +78,45 @@ void testExpire(boolean success) throws Exception {
       assertThat(deletes).isNull();
     }
   }
+
+  @ParameterizedTest
+  @ValueSource(booleans = {true, false})
+  void testCleanExpiredMetadata(boolean cleanExpiredMetadata) throws Exception 
{
+    Table table = createTable();
+    insert(table, 1, "a");
+    table.updateSchema().addColumn("extra", Types.StringType.get()).commit();
+    insert(table, 2, "b", "x");
+
+    assertThat(table.schemas().size()).isEqualTo(2);
+
+    List<TaskResult> actual;
+    Queue<StreamRecord<String>> deletes;
+    try (OneInputStreamOperatorTestHarness<Trigger, TaskResult> testHarness =
+        ProcessFunctionTestHarnesses.forProcessFunction(
+            new ExpireSnapshotsProcessor(tableLoader(), 0L, 1, 10, 
cleanExpiredMetadata))) {
+      testHarness.open();
+
+      testHarness.processElement(Trigger.create(10, 11), 
System.currentTimeMillis());
+      deletes = 
testHarness.getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM);
+      actual = testHarness.extractOutputValues();
+    }
+
+    assertThat(actual).hasSize(1);
+    TaskResult result = actual.get(0);
+    assertThat(result.startEpoch()).isEqualTo(10);
+    assertThat(result.taskIndex()).isEqualTo(11);
+    assertThat(result.success()).isEqualTo(true);
+    assertThat(result.exceptions()).isNotNull().isEmpty();
+
+    table.refresh();
+    Set<Snapshot> snapshots = Sets.newHashSet(table.snapshots());
+    assertThat(snapshots).hasSize(1);
+    assertThat(deletes).hasSize(1);
+
+    if (cleanExpiredMetadata) {
+      assertThat(table.schemas().values()).containsExactly(table.schema());
+    } else {
+      assertThat(table.schemas().size()).isEqualTo(2);

Review Comment:
   same as above



##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java:
##########
@@ -46,6 +46,7 @@ public static class Builder extends 
MaintenanceTaskBuilder<ExpireSnapshots.Build
     private Integer numSnapshots = null;
     private Integer planningWorkerPoolSize;
     private int deleteBatchSize = DELETE_BATCH_SIZE_DEFAULT;
+    private Boolean cleanExpiredMetadata = null;

Review Comment:
   we may want to make this a primitive



##########
flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java:
##########
@@ -77,4 +78,45 @@ void testExpire(boolean success) throws Exception {
       assertThat(deletes).isNull();
     }
   }
+
+  @ParameterizedTest
+  @ValueSource(booleans = {true, false})
+  void testCleanExpiredMetadata(boolean cleanExpiredMetadata) throws Exception 
{
+    Table table = createTable();
+    insert(table, 1, "a");
+    table.updateSchema().addColumn("extra", Types.StringType.get()).commit();
+    insert(table, 2, "b", "x");
+
+    assertThat(table.schemas().size()).isEqualTo(2);

Review Comment:
   ```suggestion
       assertThat(table.schemas()).hasSize(2);
   ```



##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java:
##########
@@ -52,20 +52,23 @@ public class ExpireSnapshotsProcessor extends 
ProcessFunction<Trigger, TaskResul
   private final Long maxSnapshotAgeMs;
   private final Integer numSnapshots;
   private final Integer plannerPoolSize;
+  private final Boolean cleanExpiredMetadata;
   private transient ExecutorService plannerPool;
   private transient Table table;
 
   public ExpireSnapshotsProcessor(
       TableLoader tableLoader,
       Long maxSnapshotAgeMs,
       Integer numSnapshots,
-      Integer plannerPoolSize) {
-    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+      Integer plannerPoolSize,
+      Boolean cleanExpiredMetadata) {

Review Comment:
   in Spark we made this a primitive, so maybe we should do the same 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-unsubscr...@iceberg.apache.org

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


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

Reply via email to