RussellSpitzer commented on a change in pull request #3724:
URL: https://github.com/apache/iceberg/pull/3724#discussion_r787015510
##########
File path:
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java
##########
@@ -234,6 +234,139 @@ public void testBinPackWithDeletes() throws Exception {
Assert.assertEquals("7 rows are removed", total - 7, actualRecords.size());
}
+ @Test
+ public void testBinPackWithDeleteAllData() {
+ Map<String, String> options = Maps.newHashMap();
+ options.put(TableProperties.FORMAT_VERSION, "2");
+ Table table = createTablePartitioned(1, 1, options);
+ shouldHaveFiles(table, 1);
+ table.refresh();
+
+ CloseableIterable<FileScanTask> tasks = table.newScan().planFiles();
+ List<DataFile> dataFiles =
Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file));
+ int total = (int)
dataFiles.stream().mapToLong(ContentFile::recordCount).sum();
+
+ RowDelta rowDelta = table.newRowDelta();
+ // remove all data
+ writePosDeletesToFile(table, dataFiles.get(0), total)
+ .forEach(rowDelta::addDeletes);
+
+ rowDelta.commit();
+ table.refresh();
+ List<Object[]> expectedRecords = currentData();
+ Result result = actions().rewriteDataFiles(table)
+ .option(BinPackStrategy.MIN_FILE_SIZE_BYTES, "0")
+ .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES,
Long.toString(Long.MAX_VALUE - 1))
+ .option(BinPackStrategy.MAX_FILE_SIZE_BYTES,
Long.toString(Long.MAX_VALUE))
+ .option(BinPackStrategy.DELETE_FILE_THRESHOLD, "1")
+ .option(RewriteDataFiles.USE_STARTING_SEQUENCE_NUMBER, "false")
+ .execute();
+ Assert.assertEquals("Action should rewrite 1 data files", 1,
result.rewrittenDataFilesCount());
+
+ List<Object[]> actualRecords = currentData();
+ assertEquals("Rows must match", expectedRecords, actualRecords);
+ Assert.assertEquals(
+ "Data manifest should not have existing data file",
+ 0,
+ (long)
table.currentSnapshot().dataManifests().get(0).existingFilesCount());
+ Assert.assertEquals("Data manifest should have 1 delete data file",
+ 1L,
+ (long)
table.currentSnapshot().dataManifests().get(0).deletedFilesCount());
+ Assert.assertEquals(
+ "Delete manifest added row count should equal total count",
+ total,
+ (long)
table.currentSnapshot().deleteManifests().get(0).addedRowsCount());
+ }
+
+ @Test
+ public void testBinPackWithDeleteAllDataAndSeqNum() {
+ Map<String, String> options = Maps.newHashMap();
+ options.put(TableProperties.FORMAT_VERSION, "2");
+ Table table = createTablePartitioned(1, 1, options);
+ shouldHaveFiles(table, 1);
+ table.refresh();
+
+ CloseableIterable<FileScanTask> tasks = table.newScan().planFiles();
+ List<DataFile> dataFiles =
Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file));
+ int total = (int)
dataFiles.stream().mapToLong(ContentFile::recordCount).sum();
+
+ RowDelta rowDelta = table.newRowDelta();
+ // remove all data
+ writePosDeletesToFile(table, dataFiles.get(0), total)
+ .forEach(rowDelta::addDeletes);
+
+ rowDelta.commit();
+ table.refresh();
+ List<Object[]> expectedRecords = currentData();
+ Result result = actions().rewriteDataFiles(table)
+ .option(BinPackStrategy.MIN_FILE_SIZE_BYTES, "0")
+ .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES,
Long.toString(Long.MAX_VALUE - 1))
+ .option(BinPackStrategy.MAX_FILE_SIZE_BYTES,
Long.toString(Long.MAX_VALUE))
+ .option(BinPackStrategy.DELETE_FILE_THRESHOLD, "1")
+ .option(RewriteDataFiles.USE_STARTING_SEQUENCE_NUMBER, "true")
+ .execute();
+ Assert.assertEquals("Action should rewrite 1 data files", 1,
result.rewrittenDataFilesCount());
+
+ List<Object[]> actualRecords = currentData();
+ assertEquals("Rows must match", expectedRecords, actualRecords);
+ Assert.assertEquals(
+ "Data manifest should not have existing data file",
+ 0,
+ (long)
table.currentSnapshot().dataManifests().get(0).existingFilesCount());
+ Assert.assertEquals("Data manifest should have 1 delete data file",
+ 1L,
+ (long)
table.currentSnapshot().dataManifests().get(0).deletedFilesCount());
+ Assert.assertEquals(
+ "Delete manifest added row count should equal total count",
+ total,
+ (long)
table.currentSnapshot().deleteManifests().get(0).addedRowsCount());
+ }
+
+ @Test
+ public void testBinPackPartialCommitWithDeleteAllDataAndSeqNum() {
+ Map<String, String> options = Maps.newHashMap();
+ options.put(TableProperties.FORMAT_VERSION, "2");
+ Table table = createTablePartitioned(1, 1, options);
Review comment:
Since this is only creating a single file, shouldn't this be basically
the same as the other tests with a single file being completely deleted?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]