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 2021/03/03 23:25:20 UTC

[GitHub] [iceberg] edgarRd commented on a change in pull request #2293: Spark: Fix RewriteDataFilesAction with output spec in writer

edgarRd commented on a change in pull request #2293:
URL: https://github.com/apache/iceberg/pull/2293#discussion_r586874413



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestRewriteDataFilesAction.java
##########
@@ -377,4 +378,101 @@ private void writeDF(Dataset<Row> df) {
         .mode("append")
         .save(tableLocation);
   }
+
+  @Test
+  public void testRewriteToOutputPartitionSpec() {
+    PartitionSpec spec = PartitionSpec.builderFor(SCHEMA)
+        .identity("c1")
+        .build();
+    Map<String, String> options = Maps.newHashMap();
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    table.updateSpec().addField(Expressions.truncate("c2", 2)).commit();
+
+    Assert.assertEquals("Should have 2 partitions specs", 2, table.specs().size());
+
+    List<ThreeColumnRecord> records1 = Lists.newArrayList(
+        new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA"),
+        new ThreeColumnRecord(1, "AAAAAAAAAA", "CCCC")
+    );
+    writeRecords(records1);
+
+    List<ThreeColumnRecord> records2 = Lists.newArrayList(
+        new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB"),
+        new ThreeColumnRecord(1, "BBBBBBBBBB", "DDDD")
+    );
+    writeRecords(records2);
+
+    List<ThreeColumnRecord> records3 = Lists.newArrayList(
+        new ThreeColumnRecord(2, "AAAAAAAAAA", "EEEE"),
+        new ThreeColumnRecord(2, "AAAAAAAAAA", "GGGG")
+    );
+    writeRecords(records3);
+
+    List<ThreeColumnRecord> records4 = Lists.newArrayList(
+        new ThreeColumnRecord(2, "BBBBBBBBBB", "FFFF"),
+        new ThreeColumnRecord(2, "BBBBBBBBBB", "HHHH")
+    );
+    writeRecords(records4);
+
+    table.refresh();
+
+    CloseableIterable<FileScanTask> tasks = table.newScan().planFiles();
+    List<DataFile> dataFiles = Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file));
+    Assert.assertEquals("Should have 8 data files before rewrite", 8, dataFiles.size());
+
+    Actions actions = Actions.forTable(table);
+    RewriteDataFilesActionResult result = actions
+        .rewriteDataFiles()
+        .outputSpecId(0)
+        .execute();
+    Assert.assertEquals("Action should rewrite 8 data files", 8, result.deletedDataFiles().size());
+    Assert.assertEquals("Action should add 2 data file", 2, result.addedDataFiles().size());
+
+    table.refresh();
+
+    CloseableIterable<FileScanTask> tasks2 = table.newScan().planFiles();
+    List<DataFile> dataFiles2 = Lists.newArrayList(CloseableIterable.transform(tasks2, FileScanTask::file));
+    Assert.assertEquals("Should have 2 data files after rewrite", 2, dataFiles2.size());
+
+    // Should still have all the same data
+    List<ThreeColumnRecord> expectedRecords = Lists.newArrayList();
+    expectedRecords.addAll(records1);
+    expectedRecords.addAll(records2);
+    expectedRecords.addAll(records3);
+    expectedRecords.addAll(records4);
+
+    Dataset<Row> resultDF = spark.read().format("iceberg").load(tableLocation);
+    List<ThreeColumnRecord> actualRecords = resultDF.sort("c1", "c2", "c3")
+        .as(Encoders.bean(ThreeColumnRecord.class))
+        .collectAsList();
+
+    Assert.assertEquals("Rows must match", expectedRecords, actualRecords);
+
+    List<ThreeColumnRecord> expectedFilteredRecords = Lists.newArrayList(records2);

Review comment:
       Interestingly, after rewriting the filter as a string (`c1 = 1 AND c2 = 'BBBBBBBBBB'`) to push it down the test fails, e.g.:
   This block:
   ```java
   Dataset<Row> resultDF = spark.read().format("iceberg").load(tableLocation);
   List<ThreeColumnRecord> actualFilteredRecords = resultDF.sort("c1", "c2", "c3")
       .filter("c1 = 1 AND c2 = 'BBBBBBBBBB'")
       .as(Encoders.bean(ThreeColumnRecord.class))
       .collectAsList();
   Assert.assertEquals("Rows must match", records2, actualFilteredRecords);
   ```
   works before but fails after the rewrite, returning no results.




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