pvary commented on code in PR #11497:
URL: https://github.com/apache/iceberg/pull/11497#discussion_r2073284262


##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.maintenance.operator;
+
+import static 
org.apache.iceberg.actions.SizeBasedFileRewritePlanner.MIN_INPUT_FILES;
+import static 
org.apache.iceberg.flink.maintenance.operator.RewriteUtil.newDataFiles;
+import static 
org.apache.iceberg.flink.maintenance.operator.RewriteUtil.planDataFileRewrite;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.junit.jupiter.api.Test;
+
+class TestDataFileRewritePlanner extends OperatorTestBase {
+  @Test
+  void testUnpartitioned() throws Exception {
+    Set<DataFile> expected = Sets.newHashSetWithExpectedSize(3);
+    Table table = createTable();
+    insert(table, 1, "a");
+    expected.addAll(newDataFiles(table));
+    insert(table, 2, "b");
+    expected.addAll(newDataFiles(table));
+    insert(table, 3, "c");
+    expected.addAll(newDataFiles(table));
+
+    List<DataFileRewritePlanner.PlannedGroup> actual = 
planDataFileRewrite(tableLoader());
+
+    assertThat(actual).hasSize(1);
+    assertRewriteFileGroup(actual.get(0), table, expected);
+  }
+
+  @Test
+  void testPartitioned() throws Exception {
+    Set<DataFile> expectedP1 = Sets.newHashSetWithExpectedSize(2);
+    Set<DataFile> expectedP2 = Sets.newHashSetWithExpectedSize(2);
+    Table table = createPartitionedTable();
+    insertPartitioned(table, 1, "p1");
+    expectedP1.addAll(newDataFiles(table));
+    insertPartitioned(table, 2, "p1");
+    expectedP1.addAll(newDataFiles(table));
+
+    insertPartitioned(table, 3, "p2");
+    expectedP2.addAll(newDataFiles(table));
+    insertPartitioned(table, 4, "p2");
+    expectedP2.addAll(newDataFiles(table));
+
+    // This should not participate in compaction, as there is no more files in 
the partition
+    insertPartitioned(table, 5, "p3");
+
+    List<DataFileRewritePlanner.PlannedGroup> actual = 
planDataFileRewrite(tableLoader());
+
+    assertThat(actual).hasSize(2);
+    if (actual.get(0).group().info().partition().get(0, 
String.class).equals("p1")) {
+      assertRewriteFileGroup(actual.get(0), table, expectedP1);
+      assertRewriteFileGroup(actual.get(1), table, expectedP2);
+    } else {
+      assertRewriteFileGroup(actual.get(0), table, expectedP2);
+      assertRewriteFileGroup(actual.get(1), table, expectedP1);
+    }
+  }
+
+  @Test
+  void testError() throws Exception {
+    Table table = createTable();
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+
+    try (OneInputStreamOperatorTestHarness<Trigger, 
DataFileRewritePlanner.PlannedGroup>
+        testHarness =
+            ProcessFunctionTestHarnesses.forProcessFunction(
+                new DataFileRewritePlanner(
+                    OperatorTestBase.DUMMY_TABLE_NAME,
+                    OperatorTestBase.DUMMY_TABLE_NAME,
+                    0,
+                    tableLoader(),
+                    11,
+                    1L,
+                    ImmutableMap.of(MIN_INPUT_FILES, "2")))) {
+      testHarness.open();
+
+      // Cause an exception
+      dropTable();
+
+      
assertThat(testHarness.getSideOutput(TaskResultAggregator.ERROR_STREAM)).isNull();
+      trigger(testHarness);
+      
assertThat(testHarness.getSideOutput(TaskResultAggregator.ERROR_STREAM)).hasSize(1);
+      assertThat(
+              testHarness
+                  .getSideOutput(TaskResultAggregator.ERROR_STREAM)
+                  .poll()
+                  .getValue()
+                  .getMessage())
+          .contains("Table does not exist: ");
+    }
+  }
+
+  @Test
+  void testV2Table() throws Exception {
+    Table table = createTableWithDelete();
+    update(table, 1, null, "a", "b");
+    update(table, 1, "b", "c");
+
+    List<DataFileRewritePlanner.PlannedGroup> actual = 
planDataFileRewrite(tableLoader());
+
+    assertThat(actual).hasSize(1);
+    List<FileScanTask> tasks = actual.get(0).group().fileScanTasks();
+    assertThat(tasks).hasSize(2);
+    // Find the task with the deletes
+    FileScanTask withDelete = tasks.get(0).deletes().isEmpty() ? tasks.get(1) 
: tasks.get(0);
+    assertThat(withDelete.deletes()).hasSize(2);
+    // Find the equality delete and the positional delete
+    if (withDelete.deletes().get(0).content() == FileContent.EQUALITY_DELETES) 
{
+      
assertThat(withDelete.deletes().get(1).content()).isEqualTo(FileContent.POSITION_DELETES);
+    } else {
+      
assertThat(withDelete.deletes().get(0).content()).isEqualTo(FileContent.POSITION_DELETES);
+      
assertThat(withDelete.deletes().get(1).content()).isEqualTo(FileContent.EQUALITY_DELETES);
+    }
+  }
+
+  @Test
+  void testMaxRewriteBytes() throws Exception {
+    Table table = createPartitionedTable();
+    insertPartitioned(table, 1, "p1");
+    insertPartitioned(table, 2, "p1");
+    insertPartitioned(table, 3, "p2");
+    insertPartitioned(table, 4, "p2");
+
+    // First run with high limit
+    List<DataFileRewritePlanner.PlannedGroup> planWithNoLimit = 
planDataFileRewrite(tableLoader());
+    assertThat(planWithNoLimit).hasSize(2);
+
+    // Second run with limit
+    long limit =

Review Comment:
   When the plan reaches the limit, then no consecutive group should be added 
to the plan.
   
   The `maxRewriteBytes` is added when we found not maintained tables, and we 
wanted to start running compaction on them. In this case, the first run is a 
whole table rewrite. If the table is big enough we can face resource issues. To 
solution was to run the compaction for limited data only. With the limit the 
user can configure a maximum size for the task, and consecutive runs 
incrementally could compact the remaining (not yet compacted) files.
   
   Added some comments to the 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: 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