From a8f468d6ddd0e031d41f7d07ae710511f96cbf36 Mon Sep 17 00:00:00 2001 From: Chinmay Bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Fri, 29 Dec 2023 15:22:25 +0530 Subject: [PATCH] Spark 3.5: Migrate tests to JUnit5 in actions directory (#9367) --- .../spark/actions/TestCreateActions.java | 383 ++++++++------- .../TestDeleteReachableFilesAction.java | 105 ++-- .../actions/TestExpireSnapshotsAction.java | 454 +++++++++--------- .../actions/TestRemoveOrphanFilesAction.java | 200 ++++---- .../actions/TestRemoveOrphanFilesAction3.java | 45 +- .../actions/TestRewriteDataFilesAction.java | 295 ++++++------ .../actions/TestRewriteManifestsAction.java | 235 +++++---- .../TestRewritePositionDeleteFilesAction.java | 310 ++++++------ 8 files changed, 1030 insertions(+), 997 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index 1edf57e70d33..db77da449e5d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -18,10 +18,14 @@ */ package org.apache.iceberg.spark.actions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.io.File; import java.io.FilenameFilter; import java.io.IOException; import java.net.URI; +import java.nio.file.Files; import java.nio.file.Paths; import java.util.Arrays; import java.util.Collections; @@ -31,6 +35,8 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -40,9 +46,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkCatalog; -import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkSessionCatalog; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.iceberg.spark.source.SparkTable; @@ -68,20 +74,15 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; import scala.Option; import scala.Some; import scala.collection.JavaConverters; import scala.collection.Seq; -public class TestCreateActions extends SparkCatalogTestBase { +public class TestCreateActions extends CatalogTestBase { private static final String CREATE_PARTITIONED_PARQUET = "CREATE TABLE %s (id INT, data STRING) " + "using parquet PARTITIONED BY (id) LOCATION '%s'"; private static final String CREATE_PARQUET = @@ -94,7 +95,7 @@ public class TestCreateActions extends SparkCatalogTestBase { private static final String NAMESPACE = "default"; - @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, type = {3}") public static Object[][] parameters() { return new Object[][] { new Object[] { @@ -106,7 +107,8 @@ public static Object[][] parameters() { "parquet-enabled", "true", "cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync - ) + ), + "hive" }, new Object[] { "spark_catalog", @@ -117,47 +119,48 @@ public static Object[][] parameters() { "parquet-enabled", "true", "cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync - ) + ), + "hadoop" }, new Object[] { "testhive", SparkCatalog.class.getName(), ImmutableMap.of( "type", "hive", - "default-namespace", "default") + "default-namespace", "default"), + "hive" }, new Object[] { "testhadoop", SparkCatalog.class.getName(), ImmutableMap.of( "type", "hadoop", - "default-namespace", "default") + "default-namespace", "default"), + "hadoop" } }; } - @Rule public TemporaryFolder temp = new TemporaryFolder(); - private String baseTableName = "baseTable"; private File tableDir; private String tableLocation; - private final String type; - private final TableCatalog catalog; - public TestCreateActions(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - this.catalog = (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); - this.type = config.get("type"); - } + @Parameter(index = 3) + private String type; - @Before + private TableCatalog catalog; + + @BeforeEach + @Override public void before() { + super.before(); try { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); } catch (IOException e) { throw new RuntimeException(e); } this.tableLocation = tableDir.toURI().toString(); + this.catalog = (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); spark.conf().set("hive.exec.dynamic.partition", "true"); spark.conf().set("hive.exec.dynamic.partition.mode", "nonstrict"); @@ -179,31 +182,33 @@ public void before() { .saveAsTable(baseTableName); } - @After + @AfterEach public void after() throws IOException { // Drop the hive table. spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); } - @Test + @TestTemplate public void testMigratePartitioned() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); String source = sourceName("test_migrate_partitioned_table"); String dest = source; createSourceTable(CREATE_PARTITIONED_PARQUET, source); assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); } - @Test + @TestTemplate public void testPartitionedTableWithUnRecoveredPartitions() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); String source = sourceName("test_unrecovered_partitions"); String dest = source; - File location = temp.newFolder(); + File location = Files.createTempDirectory(temp, "junit").toFile(); sql(CREATE_PARTITIONED_PARQUET, source, location); // Data generation and partition addition @@ -219,15 +224,16 @@ public void testPartitionedTableWithUnRecoveredPartitions() throws Exception { assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); } - @Test + @TestTemplate public void testPartitionedTableWithCustomPartitions() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); String source = sourceName("test_custom_parts"); String dest = source; - File tblLocation = temp.newFolder(); - File partitionDataLoc = temp.newFolder(); + File tblLocation = Files.createTempDirectory(temp, "junit").toFile(); + File partitionDataLoc = Files.createTempDirectory(temp, "junit").toFile(); // Data generation and partition addition spark.sql(String.format(CREATE_PARTITIONED_PARQUET, source, tblLocation)); @@ -243,11 +249,12 @@ public void testPartitionedTableWithCustomPartitions() throws Exception { assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); } - @Test + @TestTemplate public void testAddColumnOnMigratedTableAtEnd() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); String source = sourceName("test_add_column_migrated_table"); String dest = source; createSourceTable(CREATE_PARQUET, source); @@ -264,30 +271,31 @@ public void testAddColumnOnMigratedTableAtEnd() throws Exception { String newCol1 = "newCol1"; sparkTable.table().updateSchema().addColumn(newCol1, Types.IntegerType.get()).commit(); Schema afterSchema = table.schema(); - Assert.assertNull(beforeSchema.findField(newCol1)); - Assert.assertNotNull(afterSchema.findField(newCol1)); + assertThat(beforeSchema.findField(newCol1)).isNull(); + assertThat(afterSchema.findField(newCol1)).isNotNull(); // reads should succeed without any exceptions List results1 = sql("select * from %s order by id", dest); - Assert.assertFalse(results1.isEmpty()); + assertThat(results1).isNotEmpty(); assertEquals("Output must match", results1, expected1); String newCol2 = "newCol2"; sql("ALTER TABLE %s ADD COLUMN %s INT", dest, newCol2); StructType schema = spark.table(dest).schema(); - Assert.assertTrue(Arrays.asList(schema.fieldNames()).contains(newCol2)); + assertThat(schema.fieldNames()).contains(newCol2); // reads should succeed without any exceptions List results2 = sql("select * from %s order by id", dest); - Assert.assertFalse(results2.isEmpty()); + assertThat(results2).isNotEmpty(); assertEquals("Output must match", results2, expected2); } - @Test + @TestTemplate public void testAddColumnOnMigratedTableAtMiddle() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); String source = sourceName("test_add_column_migrated_table_middle"); String dest = source; createSourceTable(CREATE_PARQUET, source); @@ -308,26 +316,27 @@ public void testAddColumnOnMigratedTableAtMiddle() throws Exception { .moveAfter(newCol1, "id") .commit(); Schema afterSchema = table.schema(); - Assert.assertNull(beforeSchema.findField(newCol1)); - Assert.assertNotNull(afterSchema.findField(newCol1)); + assertThat(beforeSchema.findField(newCol1)).isNull(); + assertThat(afterSchema.findField(newCol1)).isNotNull(); // reads should succeed List results = sql("select * from %s order by id", dest); - Assert.assertFalse(results.isEmpty()); + assertThat(results).isNotEmpty(); assertEquals("Output must match", results, expected); } - @Test + @TestTemplate public void removeColumnsAtEnd() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); String source = sourceName("test_remove_column_migrated_table"); String dest = source; String colName1 = "newCol1"; String colName2 = "newCol2"; - File location = temp.newFolder(); + File location = Files.createTempDirectory(temp, "junit").toFile(); spark .range(10) .selectExpr("cast(id as INT)", "CAST(id as INT) " + colName1, "CAST(id as INT) " + colName2) @@ -346,29 +355,30 @@ public void removeColumnsAtEnd() throws Exception { Schema beforeSchema = table.schema(); sparkTable.table().updateSchema().deleteColumn(colName1).commit(); Schema afterSchema = table.schema(); - Assert.assertNotNull(beforeSchema.findField(colName1)); - Assert.assertNull(afterSchema.findField(colName1)); + assertThat(beforeSchema.findField(colName1)).isNotNull(); + assertThat(afterSchema.findField(colName1)).isNull(); // reads should succeed without any exceptions List results1 = sql("select * from %s order by id", dest); - Assert.assertFalse(results1.isEmpty()); + assertThat(results1).isNotEmpty(); assertEquals("Output must match", expected1, results1); sql("ALTER TABLE %s DROP COLUMN %s", dest, colName2); StructType schema = spark.table(dest).schema(); - Assert.assertFalse(Arrays.asList(schema.fieldNames()).contains(colName2)); + assertThat(schema.fieldNames()).doesNotContain(colName2); // reads should succeed without any exceptions List results2 = sql("select * from %s order by id", dest); - Assert.assertFalse(results2.isEmpty()); + assertThat(results2).isNotEmpty(); assertEquals("Output must match", expected2, results2); } - @Test + @TestTemplate public void removeColumnFromMiddle() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); String source = sourceName("test_remove_column_migrated_table_from_middle"); String dest = source; String dropColumnName = "col1"; @@ -388,31 +398,32 @@ public void removeColumnFromMiddle() throws Exception { // drop column sql("ALTER TABLE %s DROP COLUMN %s", dest, "col1"); StructType schema = spark.table(dest).schema(); - Assert.assertFalse(Arrays.asList(schema.fieldNames()).contains(dropColumnName)); + assertThat(schema.fieldNames()).doesNotContain(dropColumnName); // reads should return same output as that of non-iceberg table List results = sql("select * from %s order by id", dest); - Assert.assertFalse(results.isEmpty()); + assertThat(results).isNotEmpty(); assertEquals("Output must match", expected, results); } - @Test + @TestTemplate public void testMigrateUnpartitioned() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); String source = sourceName("test_migrate_unpartitioned_table"); String dest = source; createSourceTable(CREATE_PARQUET, source); assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); } - @Test + @TestTemplate public void testSnapshotPartitioned() throws Exception { - Assume.assumeTrue( - "Cannot snapshot with arbitrary location in a hadoop based catalog", - !type.equals("hadoop")); - File location = temp.newFolder(); + assumeThat(type) + .as("Cannot snapshot with arbitrary location in a hadoop based catalog") + .isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); String source = sourceName("test_snapshot_partitioned_table"); String dest = destName("iceberg_snapshot_partitioned"); createSourceTable(CREATE_PARTITIONED_PARQUET, source); @@ -423,12 +434,12 @@ public void testSnapshotPartitioned() throws Exception { assertIsolatedSnapshot(source, dest); } - @Test + @TestTemplate public void testSnapshotUnpartitioned() throws Exception { - Assume.assumeTrue( - "Cannot snapshot with arbitrary location in a hadoop based catalog", - !type.equals("hadoop")); - File location = temp.newFolder(); + assumeThat(type) + .as("Cannot snapshot with arbitrary location in a hadoop based catalog") + .isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); String source = sourceName("test_snapshot_unpartitioned_table"); String dest = destName("iceberg_snapshot_unpartitioned"); createSourceTable(CREATE_PARQUET, source); @@ -439,12 +450,12 @@ public void testSnapshotUnpartitioned() throws Exception { assertIsolatedSnapshot(source, dest); } - @Test + @TestTemplate public void testSnapshotHiveTable() throws Exception { - Assume.assumeTrue( - "Cannot snapshot with arbitrary location in a hadoop based catalog", - !type.equals("hadoop")); - File location = temp.newFolder(); + assumeThat(type) + .as("Cannot snapshot with arbitrary location in a hadoop based catalog") + .isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); String source = sourceName("snapshot_hive_table"); String dest = destName("iceberg_snapshot_hive_table"); createSourceTable(CREATE_HIVE_EXTERNAL_PARQUET, source); @@ -455,19 +466,19 @@ public void testSnapshotHiveTable() throws Exception { assertIsolatedSnapshot(source, dest); } - @Test + @TestTemplate public void testMigrateHiveTable() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); String source = sourceName("migrate_hive_table"); String dest = source; createSourceTable(CREATE_HIVE_EXTERNAL_PARQUET, source); assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); } - @Test + @TestTemplate public void testSnapshotManagedHiveTable() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - File location = temp.newFolder(); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); String source = sourceName("snapshot_managed_hive_table"); String dest = destName("iceberg_snapshot_managed_hive_table"); createSourceTable(CREATE_HIVE_PARQUET, source); @@ -478,10 +489,10 @@ public void testSnapshotManagedHiveTable() throws Exception { assertIsolatedSnapshot(source, dest); } - @Test + @TestTemplate public void testMigrateManagedHiveTable() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - File location = temp.newFolder(); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); String source = sourceName("migrate_managed_hive_table"); String dest = destName("iceberg_migrate_managed_hive_table"); createSourceTable(CREATE_HIVE_PARQUET, source); @@ -491,7 +502,7 @@ public void testMigrateManagedHiveTable() throws Exception { dest); } - @Test + @TestTemplate public void testProperties() throws Exception { String source = sourceName("test_properties_table"); String dest = destName("iceberg_properties"); @@ -516,17 +527,13 @@ public void testProperties() throws Exception { expectedProps.put("dogs", "sundance"); for (Map.Entry entry : expectedProps.entrySet()) { - Assert.assertTrue( - "Created table missing property " + entry.getKey(), - table.properties().containsKey(entry.getKey())); - Assert.assertEquals( - "Property value is not the expected value", - entry.getValue(), - table.properties().get(entry.getKey())); + assertThat(table.properties()) + .as("Property value is not the expected value") + .containsEntry(entry.getKey(), entry.getValue()); } } - @Test + @TestTemplate public void testSparkTableReservedProperties() throws Exception { String destTableName = "iceberg_reserved_properties"; String source = sourceName("test_reserved_properties_table"); @@ -542,28 +549,36 @@ public void testSparkTableReservedProperties() throws Exception { String[] keys = {"provider", "format", "current-snapshot-id", "location", "sort-order"}; for (String entry : keys) { - Assert.assertTrue( - "Created table missing reserved property " + entry, - table.properties().containsKey(entry)); + assertThat(table.properties()) + .as("Created table missing reserved property " + entry) + .containsKey(entry); } - Assert.assertEquals("Unexpected provider", "iceberg", table.properties().get("provider")); - Assert.assertEquals("Unexpected format", "iceberg/parquet", table.properties().get("format")); - Assert.assertNotEquals( - "No current-snapshot-id found", "none", table.properties().get("current-snapshot-id")); - Assert.assertTrue( - "Location isn't correct", table.properties().get("location").endsWith(destTableName)); - - Assert.assertEquals("Unexpected format-version", "1", table.properties().get("format-version")); + assertThat(table.properties().get("provider")).as("Unexpected provider").isEqualTo("iceberg"); + assertThat(table.properties().get("format")) + .as("Unexpected provider") + .isEqualTo("iceberg/parquet"); + assertThat(table.properties().get("current-snapshot-id")) + .as("No current-snapshot-id found") + .isNotEqualTo("none"); + assertThat(table.properties().get("location")) + .as("Location isn't correct") + .endsWith(destTableName); + + assertThat(table.properties().get("format-version")) + .as("Unexpected format-version") + .isEqualTo("1"); table.table().updateProperties().set("format-version", "2").commit(); - Assert.assertEquals("Unexpected format-version", "2", table.properties().get("format-version")); + assertThat(table.properties().get("format-version")) + .as("Unexpected format-version") + .isEqualTo("2"); - Assert.assertEquals( - "Sort-order isn't correct", - "id ASC NULLS FIRST, data DESC NULLS LAST", - table.properties().get("sort-order")); - Assert.assertNull( - "Identifier fields should be null", table.properties().get("identifier-fields")); + assertThat(table.properties().get("sort-order")) + .as("Sort-order isn't correct") + .isEqualTo("id ASC NULLS FIRST, data DESC NULLS LAST"); + assertThat(table.properties().get("identifier-fields")) + .as("Identifier fields should be null") + .isNull(); table .table() @@ -572,11 +587,12 @@ public void testSparkTableReservedProperties() throws Exception { .requireColumn("id") .setIdentifierFields("id") .commit(); - Assert.assertEquals( - "Identifier fields aren't correct", "[id]", table.properties().get("identifier-fields")); + assertThat(table.properties().get("identifier-fields")) + .as("Identifier fields aren't correct") + .isEqualTo("[id]"); } - @Test + @TestTemplate public void testSnapshotDefaultLocation() throws Exception { String source = sourceName("test_snapshot_default"); String dest = destName("iceberg_snapshot_default"); @@ -585,13 +601,14 @@ public void testSnapshotDefaultLocation() throws Exception { assertIsolatedSnapshot(source, dest); } - @Test + @TestTemplate public void schemaEvolutionTestWithSparkAPI() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); - File location = temp.newFolder(); + File location = Files.createTempDirectory(temp, "junit").toFile(); String tblName = sourceName("schema_evolution_test"); // Data generation and partition addition @@ -639,11 +656,12 @@ public void schemaEvolutionTestWithSparkAPI() throws Exception { assertEquals("Output must match", expectedAfterAddColumn, afterMigarteAfterAddResults); } - @Test + @TestTemplate public void schemaEvolutionTestWithSparkSQL() throws Exception { - Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); - Assume.assumeTrue( - "Can only migrate from Spark Session Catalog", catalog.name().equals("spark_catalog")); + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); String tblName = sourceName("schema_evolution_test_sql"); // Data generation and partition addition @@ -686,52 +704,52 @@ public void schemaEvolutionTestWithSparkSQL() throws Exception { assertEquals("Output must match", expectedAfterAddColumn, afterMigarteAfterAddResults); } - @Test + @TestTemplate public void testHiveStyleThreeLevelList() throws Exception { threeLevelList(true); } - @Test + @TestTemplate public void testThreeLevelList() throws Exception { threeLevelList(false); } - @Test + @TestTemplate public void testHiveStyleThreeLevelListWithNestedStruct() throws Exception { threeLevelListWithNestedStruct(true); } - @Test + @TestTemplate public void testThreeLevelListWithNestedStruct() throws Exception { threeLevelListWithNestedStruct(false); } - @Test + @TestTemplate public void testHiveStyleThreeLevelLists() throws Exception { threeLevelLists(true); } - @Test + @TestTemplate public void testThreeLevelLists() throws Exception { threeLevelLists(false); } - @Test + @TestTemplate public void testHiveStyleStructOfThreeLevelLists() throws Exception { structOfThreeLevelLists(true); } - @Test + @TestTemplate public void testStructOfThreeLevelLists() throws Exception { structOfThreeLevelLists(false); } - @Test + @TestTemplate public void testTwoLevelList() throws IOException { spark.conf().set("spark.sql.parquet.writeLegacyFormat", true); String tableName = sourceName("testTwoLevelList"); - File location = temp.newFolder(); + File location = Files.createTempDirectory(temp, "junit").toFile(); StructType sparkSchema = new StructType( @@ -791,7 +809,7 @@ public boolean accept(File dir, String name) { HadoopInputFile.fromPath( new Path(parquetFile.getPath()), spark.sessionState().newHadoopConf())); MessageType schema = pqReader.getFooter().getFileMetaData().getSchema(); - Assert.assertEquals(MessageTypeParser.parseMessageType(expectedParquetSchema), schema); + assertThat(schema).isEqualTo(MessageTypeParser.parseMessageType(expectedParquetSchema)); // create sql table on top of it sql( @@ -806,7 +824,7 @@ public boolean accept(File dir, String name) { // check migrated table is returning expected result List results = sql("SELECT * FROM %s", tableName); - Assert.assertFalse(results.isEmpty()); + assertThat(results).isNotEmpty(); assertEquals("Output must match", expected, results); } @@ -814,7 +832,7 @@ private void threeLevelList(boolean useLegacyMode) throws Exception { spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelList_%s", useLegacyMode)); - File location = temp.newFolder(); + File location = Files.createTempDirectory(temp, "junit").toFile(); sql( "CREATE TABLE %s (col1 ARRAY>)" + " STORED AS parquet" + " LOCATION '%s'", tableName, location); @@ -828,7 +846,7 @@ private void threeLevelList(boolean useLegacyMode) throws Exception { // check migrated table is returning expected result List results = sql("SELECT * FROM %s", tableName); - Assert.assertFalse(results.isEmpty()); + assertThat(results).isNotEmpty(); assertEquals("Output must match", expected, results); } @@ -837,7 +855,7 @@ private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Except String tableName = sourceName(String.format("threeLevelListWithNestedStruct_%s", useLegacyMode)); - File location = temp.newFolder(); + File location = Files.createTempDirectory(temp, "junit").toFile(); sql( "CREATE TABLE %s (col1 ARRAY>>)" + " STORED AS parquet" @@ -853,7 +871,7 @@ private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Except // check migrated table is returning expected result List results = sql("SELECT * FROM %s", tableName); - Assert.assertFalse(results.isEmpty()); + assertThat(results).isNotEmpty(); assertEquals("Output must match", expected, results); } @@ -861,7 +879,7 @@ private void threeLevelLists(boolean useLegacyMode) throws Exception { spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelLists_%s", useLegacyMode)); - File location = temp.newFolder(); + File location = Files.createTempDirectory(temp, "junit").toFile(); sql( "CREATE TABLE %s (col1 ARRAY>, col3 ARRAY>)" + " STORED AS parquet" @@ -880,7 +898,7 @@ private void threeLevelLists(boolean useLegacyMode) throws Exception { // check migrated table is returning expected result List results = sql("SELECT * FROM %s", tableName); - Assert.assertFalse(results.isEmpty()); + assertThat(results).isNotEmpty(); assertEquals("Output must match", expected, results); } @@ -888,7 +906,7 @@ private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("structOfThreeLevelLists_%s", useLegacyMode)); - File location = temp.newFolder(); + File location = Files.createTempDirectory(temp, "junit").toFile(); sql( "CREATE TABLE %s (col1 STRUCT>>)" + " STORED AS parquet" @@ -904,7 +922,7 @@ private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { // check migrated table is returning expected result List results = sql("SELECT * FROM %s", tableName); - Assert.assertFalse(results.isEmpty()); + assertThat(results).isNotEmpty(); assertEquals("Output must match", expected, results); } @@ -925,7 +943,7 @@ private CatalogTable loadSessionTable(String name) private void createSourceTable(String createStatement, String tableName) throws IOException, NoSuchTableException, NoSuchDatabaseException, ParseException { - File location = temp.newFolder(); + File location = Files.createTempDirectory(temp, "junit").toFile(); spark.sql(String.format(createStatement, tableName, location)); CatalogTable table = loadSessionTable(tableName); String format = table.provider().get(); @@ -945,8 +963,9 @@ private void assertMigratedFileCount(MigrateTable migrateAction, String source, long expectedFiles = expectedFilesCount(source); MigrateTable.Result migratedFiles = migrateAction.execute(); validateTables(source, dest); - Assert.assertEquals( - "Expected number of migrated files", expectedFiles, migratedFiles.migratedDataFilesCount()); + assertThat(migratedFiles.migratedDataFilesCount()) + .as("Expected number of migrated files") + .isEqualTo(expectedFiles); } // Counts the number of files in the source table, makes sure the same files exist in the @@ -956,26 +975,31 @@ private void assertSnapshotFileCount(SnapshotTable snapshotTable, String source, long expectedFiles = expectedFilesCount(source); SnapshotTable.Result snapshotTableResult = snapshotTable.execute(); validateTables(source, dest); - Assert.assertEquals( - "Expected number of imported snapshot files", - expectedFiles, - snapshotTableResult.importedDataFilesCount()); + assertThat(snapshotTableResult.importedDataFilesCount()) + .as("Expected number of imported snapshot files") + .isEqualTo(expectedFiles); } private void validateTables(String source, String dest) throws NoSuchTableException, ParseException { List expected = spark.table(source).collectAsList(); SparkTable destTable = loadTable(dest); - Assert.assertEquals( - "Provider should be iceberg", - "iceberg", - destTable.properties().get(TableCatalog.PROP_PROVIDER)); + assertThat(destTable.properties().get(TableCatalog.PROP_PROVIDER)) + .as("Provider should be iceberg") + .isEqualTo("iceberg"); List actual = spark.table(dest).collectAsList(); - Assert.assertTrue( - String.format( - "Rows in migrated table did not match\nExpected :%s rows \nFound :%s", - expected, actual), - expected.containsAll(actual) && actual.containsAll(expected)); + assertThat(actual) + .as( + String.format( + "Rows in migrated table did not match\nExpected :%s rows \nFound :%s", + expected, actual)) + .containsAll(expected); + assertThat(expected) + .as( + String.format( + "Rows in migrated table did not match\nExpected :%s rows \nFound :%s", + expected, actual)) + .containsAll(actual); } private long expectedFilesCount(String source) @@ -1016,14 +1040,15 @@ private void assertIsolatedSnapshot(String source, String dest) { df.write().format("iceberg").mode("append").saveAsTable(dest); List result = spark.sql(String.format("SELECT * FROM %s", source)).collectAsList(); - Assert.assertEquals( - "No additional rows should be added to the original table", expected.size(), result.size()); + assertThat(result) + .as("No additional rows should be added to the original table") + .hasSameSizeAs(expected); List snapshot = spark .sql(String.format("SELECT * FROM %s WHERE id = 4 AND data = 'd'", dest)) .collectAsList(); - Assert.assertEquals("Added row not found in snapshot", 1, snapshot.size()); + assertThat(snapshot).as("Added row not found in snapshot").hasSize(1); } private String sourceName(String source) { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 384405265500..5126b67fc80d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -19,8 +19,11 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; +import java.nio.file.Path; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; @@ -44,20 +47,16 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestDeleteReachableFilesAction extends SparkTestBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestDeleteReachableFilesAction extends TestBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); private static final Schema SCHEMA = new Schema( @@ -113,13 +112,13 @@ public class TestDeleteReachableFilesAction extends SparkTestBase { .withRecordCount(1) .build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private Table table; - @Before + @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = temp.newFolder(); + File tableDir = temp.resolve("junit").toFile(); String tableLocation = tableDir.toURI().toString(); this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); spark.conf().set("spark.sql.shuffle.partitions", SHUFFLE_PARTITIONS); @@ -133,30 +132,29 @@ private void checkRemoveFilesResults( long expectedManifestListsDeleted, long expectedOtherFilesDeleted, DeleteReachableFiles.Result results) { - Assert.assertEquals( - "Incorrect number of manifest files deleted", - expectedManifestsDeleted, - results.deletedManifestsCount()); - Assert.assertEquals( - "Incorrect number of datafiles deleted", - expectedDatafiles, - results.deletedDataFilesCount()); - Assert.assertEquals( - "Incorrect number of position delete files deleted", - expectedPosDeleteFiles, - results.deletedPositionDeleteFilesCount()); - Assert.assertEquals( - "Incorrect number of equality delete files deleted", - expectedEqDeleteFiles, - results.deletedEqualityDeleteFilesCount()); - Assert.assertEquals( - "Incorrect number of manifest lists deleted", - expectedManifestListsDeleted, - results.deletedManifestListsCount()); - Assert.assertEquals( - "Incorrect number of other lists deleted", - expectedOtherFilesDeleted, - results.deletedOtherFilesCount()); + assertThat(results.deletedManifestsCount()) + .as("Incorrect number of manifest files deleted") + .isEqualTo(expectedManifestsDeleted); + + assertThat(results.deletedDataFilesCount()) + .as("Incorrect number of datafiles deleted") + .isEqualTo(expectedDatafiles); + + assertThat(results.deletedPositionDeleteFilesCount()) + .as("Incorrect number of position delete files deleted") + .isEqualTo(expectedPosDeleteFiles); + + assertThat(results.deletedEqualityDeleteFilesCount()) + .as("Incorrect number of equality delete files deleted") + .isEqualTo(expectedEqDeleteFiles); + + assertThat(results.deletedManifestListsCount()) + .as("Incorrect number of manifest lists deleted") + .isEqualTo(expectedManifestListsDeleted); + + assertThat(results.deletedOtherFilesCount()) + .as("Incorrect number of other lists deleted") + .isEqualTo(expectedOtherFilesDeleted); } @Test @@ -196,15 +194,17 @@ public void dataFilesCleanupWithParallelTasks() { // Verifies that the delete methods ran in the threads created by the provided ExecutorService // ThreadFactory - Assert.assertEquals( - deleteThreads, - Sets.newHashSet("remove-files-0", "remove-files-1", "remove-files-2", "remove-files-3")); + assertThat(deleteThreads) + .isEqualTo( + Sets.newHashSet( + "remove-files-0", "remove-files-1", "remove-files-2", "remove-files-3")); Lists.newArrayList(FILE_A, FILE_B, FILE_C, FILE_D) .forEach( file -> - Assert.assertTrue( - "FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()))); + assertThat(deletedFiles) + .as("FILE_A should be deleted") + .contains(FILE_A.path().toString())); checkRemoveFilesResults(4L, 0, 0, 6L, 4L, 6, result); } @@ -329,10 +329,9 @@ public void testUseLocalIterator() { checkRemoveFilesResults(3L, 0, 0, 4L, 3L, 5, results); - Assert.assertEquals( - "Expected total jobs to be equal to total number of shuffle partitions", - totalJobsRun, - SHUFFLE_PARTITIONS); + assertThat(totalJobsRun) + .as("Expected total jobs to be equal to total number of shuffle partitions") + .isEqualTo(SHUFFLE_PARTITIONS); }); } @@ -345,11 +344,10 @@ public void testIgnoreMetadataFilesNotFound() { DeleteOrphanFiles.Result result = sparkActions().deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals("Should delete 1 file", 1, Iterables.size(result.orphanFileLocations())); - Assert.assertTrue( - "Should remove v1 file", - StreamSupport.stream(result.orphanFileLocations().spliterator(), false) - .anyMatch(file -> file.contains("v1.metadata.json"))); + assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); + assertThat(StreamSupport.stream(result.orphanFileLocations().spliterator(), false)) + .as("Should remove v1 file") + .anyMatch(file -> file.contains("v1.metadata.json")); DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()); @@ -363,7 +361,7 @@ public void testEmptyIOThrowsException() { DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions().deleteReachableFiles(metadataLocation(table)).io(null); - Assertions.assertThatThrownBy(baseRemoveFilesSparkAction::execute) + assertThatThrownBy(baseRemoveFilesSparkAction::execute) .isInstanceOf(IllegalArgumentException.class) .hasMessage("File IO cannot be null"); } @@ -372,8 +370,7 @@ public void testEmptyIOThrowsException() { public void testRemoveFilesActionWhenGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); - Assertions.assertThatThrownBy( - () -> sparkActions().deleteReachableFiles(metadataLocation(table)).execute()) + assertThatThrownBy(() -> sparkActions().deleteReachableFiles(metadataLocation(table)).execute()) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot delete files: GC is disabled (deleting files may corrupt other tables)"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index d5ac1b80649a..74169b9f27f1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -19,9 +19,12 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -52,18 +55,15 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestExpireSnapshotsAction extends SparkTestBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestExpireSnapshotsAction extends TestBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); private static final Schema SCHEMA = new Schema( @@ -119,15 +119,15 @@ public class TestExpireSnapshotsAction extends SparkTestBase { .withRecordCount(1) .build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private File tableDir; private String tableLocation; private Table table; - @Before + @BeforeEach public void setupTableLocation() throws Exception { - this.tableDir = temp.newFolder(); + this.tableDir = temp.resolve("junit").toFile(); this.tableLocation = tableDir.toURI().toString(); this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); spark.conf().set("spark.sql.shuffle.partitions", SHUFFLE_PARTITIONS); @@ -153,26 +153,25 @@ private void checkExpirationResults( long expectedManifestListsDeleted, ExpireSnapshots.Result results) { - Assert.assertEquals( - "Incorrect number of manifest files deleted", - expectedManifestsDeleted, - results.deletedManifestsCount()); - Assert.assertEquals( - "Incorrect number of datafiles deleted", - expectedDatafiles, - results.deletedDataFilesCount()); - Assert.assertEquals( - "Incorrect number of pos deletefiles deleted", - expectedPosDeleteFiles, - results.deletedPositionDeleteFilesCount()); - Assert.assertEquals( - "Incorrect number of eq deletefiles deleted", - expectedEqDeleteFiles, - results.deletedEqualityDeleteFilesCount()); - Assert.assertEquals( - "Incorrect number of manifest lists deleted", - expectedManifestListsDeleted, - results.deletedManifestListsCount()); + assertThat(results.deletedManifestsCount()) + .as("Incorrect number of manifest files deleted") + .isEqualTo(expectedManifestsDeleted); + + assertThat(results.deletedDataFilesCount()) + .as("Incorrect number of datafiles deleted") + .isEqualTo(expectedDatafiles); + + assertThat(results.deletedPositionDeleteFilesCount()) + .as("Incorrect number of pos deletefiles deleted") + .isEqualTo(expectedPosDeleteFiles); + + assertThat(results.deletedEqualityDeleteFilesCount()) + .as("Incorrect number of eq deletefiles deleted") + .isEqualTo(expectedEqDeleteFiles); + + assertThat(results.deletedManifestListsCount()) + .as("Incorrect number of manifest lists deleted") + .isEqualTo(expectedManifestListsDeleted); } @Test @@ -188,8 +187,7 @@ public void testFilesCleaned() throws Exception { ExpireSnapshots.Result results = SparkActions.get().expireSnapshots(table).expireOlderThan(end).execute(); - Assert.assertEquals( - "Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Table does not have 1 snapshot after expiration").hasSize(1); checkExpirationResults(1L, 0L, 0L, 1L, 2L, results); } @@ -234,13 +232,16 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { // Verifies that the delete methods ran in the threads created by the provided ExecutorService // ThreadFactory - Assert.assertEquals( - deleteThreads, - Sets.newHashSet( - "remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3")); + assertThat(deleteThreads) + .isEqualTo( + Sets.newHashSet( + "remove-snapshot-0", + "remove-snapshot-1", + "remove-snapshot-2", + "remove-snapshot-3")); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); - Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString())); + assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.path().toString()); + assertThat(deletedFiles).as("FILE_B should be deleted").contains(FILE_B.path().toString()); checkExpirationResults(2L, 0L, 0L, 3L, 3L, result); } @@ -296,10 +297,8 @@ public void testRetainLastWithExpireOlderThan() { // Retain last 2 snapshots SparkActions.get().expireSnapshots(table).expireOlderThan(t3).retainLast(2).execute(); - Assert.assertEquals( - "Should have two snapshots.", 2, Lists.newArrayList(table.snapshots()).size()); - Assert.assertEquals( - "First snapshot should not present.", null, table.snapshot(firstSnapshotId)); + assertThat(table.snapshots()).as("Should have two snapshots.").hasSize(2); + assertThat(table.snapshot(firstSnapshotId)).as("First snapshot should not present.").isNull(); } @Test @@ -330,12 +329,9 @@ public void testExpireTwoSnapshotsById() throws Exception { .expireSnapshotId(secondSnapshotID) .execute(); - Assert.assertEquals( - "Should have one snapshots.", 1, Lists.newArrayList(table.snapshots()).size()); - Assert.assertEquals( - "First snapshot should not present.", null, table.snapshot(firstSnapshotId)); - Assert.assertEquals( - "Second snapshot should not be present.", null, table.snapshot(secondSnapshotID)); + assertThat(table.snapshots()).as("Should have one snapshot.").hasSize(1); + assertThat(table.snapshot(firstSnapshotId)).as("First snapshot should not present.").isNull(); + assertThat(table.snapshot(secondSnapshotID)).as("Second snapshot should not present.").isNull(); checkExpirationResults(0L, 0L, 0L, 0L, 2L, result); } @@ -366,10 +362,8 @@ public void testRetainLastWithExpireById() { .retainLast(3) .execute(); - Assert.assertEquals( - "Should have two snapshots.", 2, Lists.newArrayList(table.snapshots()).size()); - Assert.assertEquals( - "First snapshot should not present.", null, table.snapshot(firstSnapshotId)); + assertThat(table.snapshots()).as("Should have 2 snapshots.").hasSize(2); + assertThat(table.snapshot(firstSnapshotId)).as("First snapshot should not present.").isNull(); checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); } @@ -393,12 +387,10 @@ public void testRetainLastWithTooFewSnapshots() { ExpireSnapshots.Result result = SparkActions.get().expireSnapshots(table).expireOlderThan(t2).retainLast(3).execute(); - Assert.assertEquals( - "Should have two snapshots", 2, Lists.newArrayList(table.snapshots()).size()); - Assert.assertEquals( - "First snapshot should still present", - firstSnapshotId, - table.snapshot(firstSnapshotId).snapshotId()); + assertThat(table.snapshots()).as("Should have two snapshots.").hasSize(2); + assertThat(table.snapshot(firstSnapshotId)) + .as("First snapshot should still be present.") + .isNotNull(); checkExpirationResults(0L, 0L, 0L, 0L, 0L, result); } @@ -434,10 +426,10 @@ public void testRetainLastKeepsExpiringSnapshot() { .retainLast(2) .execute(); - Assert.assertEquals( - "Should have three snapshots.", 3, Lists.newArrayList(table.snapshots()).size()); - Assert.assertNotNull( - "Second snapshot should present.", table.snapshot(secondSnapshot.snapshotId())); + assertThat(table.snapshots()).as("Should have three snapshots.").hasSize(3); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("First snapshot should be present.") + .isNotNull(); checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); } @@ -447,7 +439,7 @@ public void testExpireSnapshotsWithDisabledGarbageCollection() { table.newAppend().appendFile(FILE_A).commit(); - Assertions.assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table)) + assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table)) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot expire snapshots: GC is disabled (deleting files may corrupt other tables)"); @@ -482,10 +474,10 @@ public void testExpireOlderThanMultipleCalls() { .expireOlderThan(thirdSnapshot.timestampMillis()) .execute(); - Assert.assertEquals( - "Should have one snapshots.", 1, Lists.newArrayList(table.snapshots()).size()); - Assert.assertNull( - "Second snapshot should not present.", table.snapshot(secondSnapshot.snapshotId())); + assertThat(table.snapshots()).as("Should have one snapshot.").hasSize(1); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Second snapshot should not present.") + .isNull(); checkExpirationResults(0L, 0L, 0L, 0L, 2L, result); } @@ -519,17 +511,16 @@ public void testRetainLastMultipleCalls() { .retainLast(1) .execute(); - Assert.assertEquals( - "Should have one snapshots.", 1, Lists.newArrayList(table.snapshots()).size()); - Assert.assertNull( - "Second snapshot should not present.", table.snapshot(secondSnapshot.snapshotId())); + assertThat(table.snapshots()).as("Should have one snapshot.").hasSize(1); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Second snapshot should not present.") + .isNull(); checkExpirationResults(0L, 0L, 0L, 0L, 2L, result); } @Test public void testRetainZeroSnapshots() { - Assertions.assertThatThrownBy( - () -> SparkActions.get().expireSnapshots(table).retainLast(0).execute()) + assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table).retainLast(0).execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Number of snapshots to retain must be at least 1, cannot be: 0"); } @@ -553,7 +544,7 @@ public void testScanExpiredManifestInValidSnapshotAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.path().toString()); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -582,7 +573,7 @@ public void testScanExpiredManifestInValidSnapshotFastAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.path().toString()); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -639,11 +630,12 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(file.path()); } }); - Assert.assertSame( - "Files deleted count should be expected", expectedDeletes.size(), deletedFiles.size()); + assertThat(expectedDeletes) + .as("Files deleted count should be expected") + .hasSameSizeAs(deletedFiles); // Take the diff expectedDeletes.removeAll(deletedFiles); - Assert.assertTrue("Exactly same files should be deleted", expectedDeletes.isEmpty()); + assertThat(expectedDeletes).as("Exactly same files should be deleted").isEmpty(); } /** @@ -659,7 +651,7 @@ public void testWithCherryPickTableSnapshot() { // `B` commit Set deletedAFiles = Sets.newHashSet(); table.newOverwrite().addFile(FILE_B).deleteFile(FILE_A).deleteWith(deletedAFiles::add).commit(); - Assert.assertTrue("No files should be physically deleted", deletedAFiles.isEmpty()); + assertThat(deletedAFiles).as("No files should be physically deleted").isEmpty(); // pick the snapshot 'B` Snapshot snapshotB = table.currentSnapshot(); @@ -694,7 +686,7 @@ public void testWithCherryPickTableSnapshot() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + assertThat(deletedFiles).doesNotContain(item.path().toString()); }); }); @@ -743,7 +735,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + assertThat(deletedFiles).doesNotContain(item.path().toString()); }); }); checkExpirationResults(0L, 0L, 0L, 1L, 1L, firstResult); @@ -763,7 +755,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + assertThat(deletedFiles).doesNotContain(item.path().toString()); }); }); checkExpirationResults(0L, 0L, 0L, 0L, 2L, secondResult); @@ -792,16 +784,15 @@ public void testExpireOlderThan() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNull( - "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove only the expired manifest list location", - Sets.newHashSet(firstSnapshot.manifestListLocation()), - deletedFiles); + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should remove the oldest snapshot.") + .isNull(); + assertThat(deletedFiles) + .as("Should remove only the expired manifest list location.") + .isEqualTo(Sets.newHashSet(firstSnapshot.manifestListLocation())); checkExpirationResults(0, 0, 0, 0, 1, result); } @@ -811,18 +802,16 @@ public void testExpireOlderThanWithDelete() { table.newAppend().appendFile(FILE_A).commit(); Snapshot firstSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create one manifest", 1, firstSnapshot.allManifests(table.io()).size()); + assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); rightAfterSnapshot(); table.newDelete().deleteFile(FILE_A).commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create replace manifest with a rewritten manifest", - 1, - secondSnapshot.allManifests(table.io()).size()); + assertThat(secondSnapshot.allManifests(table.io())) + .as("Should create replace manifest with a rewritten manifest") + .hasSize(1); table.newAppend().appendFile(FILE_B).commit(); @@ -841,31 +830,31 @@ public void testExpireOlderThanWithDelete() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNull( - "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertNull( - "Expire should remove the second oldest snapshot", - table.snapshot(secondSnapshot.snapshotId())); - - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", - Sets.newHashSet( - firstSnapshot.manifestListLocation(), // snapshot expired - firstSnapshot - .allManifests(table.io()) - .get(0) - .path(), // manifest was rewritten for delete - secondSnapshot.manifestListLocation(), // snapshot expired - secondSnapshot - .allManifests(table.io()) - .get(0) - .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted - deletedFiles); + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should remove the oldest snapshot.") + .isNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the second oldest snapshot.") + .isNull(); + assertThat(deletedFiles) + .as("Should remove expired manifest lists and deleted data file.") + .isEqualTo( + Sets.newHashSet( + firstSnapshot.manifestListLocation(), // snapshot expired + firstSnapshot + .allManifests(table.io()) + .get(0) + .path(), // manifest was rewritten for delete + secondSnapshot.manifestListLocation(), // snapshot expired + secondSnapshot + .allManifests(table.io()) + .get(0) + .path(), // manifest contained only deletes, was dropped + FILE_A.path()) // deleted + ); checkExpirationResults(1, 0, 0, 2, 2, result); } @@ -878,8 +867,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Snapshot firstSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create one manifest", 1, firstSnapshot.allManifests(table.io()).size()); + assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); rightAfterSnapshot(); @@ -889,11 +877,9 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should replace manifest with a rewritten manifest", - 1, - secondSnapshot.allManifests(table.io()).size()); - + assertThat(secondSnapshot.allManifests(table.io())) + .as("Should replace manifest with a rewritten manifest") + .hasSize(1); table .newFastAppend() // do not merge to keep the last snapshot's manifest valid .appendFile(FILE_C) @@ -914,28 +900,28 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNull( - "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertNull( - "Expire should remove the second oldest snapshot", - table.snapshot(secondSnapshot.snapshotId())); - - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", - Sets.newHashSet( - firstSnapshot.manifestListLocation(), // snapshot expired - firstSnapshot - .allManifests(table.io()) - .get(0) - .path(), // manifest was rewritten for delete - secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted - deletedFiles); - + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should remove the oldest snapshot.") + .isNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the second oldest snapshot.") + .isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and deleted data file.") + .isEqualTo( + Sets.newHashSet( + firstSnapshot.manifestListLocation(), // snapshot expired + firstSnapshot + .allManifests(table.io()) + .get(0) + .path(), // manifest was rewritten for delete + secondSnapshot.manifestListLocation(), // snapshot expired + FILE_A.path()) // deleted + ); checkExpirationResults(1, 0, 0, 1, 2, result); } @@ -947,8 +933,7 @@ public void testExpireOlderThanWithRollback() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Snapshot firstSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create one manifest", 1, firstSnapshot.allManifests(table.io()).size()); + assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); rightAfterSnapshot(); @@ -958,8 +943,7 @@ public void testExpireOlderThanWithRollback() { Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests(table.io())); secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); - Assert.assertEquals( - "Should add one new manifest for append", 1, secondSnapshotManifests.size()); + assertThat(secondSnapshotManifests).as("Should add one new manifest for append").hasSize(1); table.manageSnapshots().rollbackTo(firstSnapshot.snapshotId()).commit(); @@ -976,23 +960,25 @@ public void testExpireOlderThanWithRollback() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNotNull( - "Expire should keep the oldest snapshot, current", - table.snapshot(firstSnapshot.snapshotId())); - Assert.assertNull( - "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", - Sets.newHashSet( - secondSnapshot.manifestListLocation(), // snapshot expired - Iterables.getOnlyElement(secondSnapshotManifests) - .path()), // manifest is no longer referenced - deletedFiles); + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should keep the oldest snapshot, current.") + .isNotNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the orphaned snapshot.") + .isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and reverted appended data file") + .isEqualTo( + Sets.newHashSet( + secondSnapshot.manifestListLocation(), // snapshot expired + Iterables.getOnlyElement(secondSnapshotManifests) + .path()) // manifest is no longer referenced + ); checkExpirationResults(0, 0, 0, 1, 1, result); } @@ -1002,9 +988,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { table.newAppend().appendFile(FILE_A).commit(); Snapshot firstSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create one manifest", 1, firstSnapshot.allManifests(table.io()).size()); - + assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); rightAfterSnapshot(); table.newAppend().appendFile(FILE_B).commit(); @@ -1013,8 +997,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests(table.io())); secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); - Assert.assertEquals( - "Should add one new manifest for append", 1, secondSnapshotManifests.size()); + assertThat(secondSnapshotManifests).as("Should add one new manifest for append").hasSize(1); table.manageSnapshots().rollbackTo(firstSnapshot.snapshotId()).commit(); @@ -1031,24 +1014,26 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNotNull( - "Expire should keep the oldest snapshot, current", - table.snapshot(firstSnapshot.snapshotId())); - Assert.assertNull( - "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", - Sets.newHashSet( - secondSnapshot.manifestListLocation(), // snapshot expired - Iterables.getOnlyElement(secondSnapshotManifests) - .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back - deletedFiles); + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should keep the oldest snapshot, current.") + .isNotNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the orphaned snapshot.") + .isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and reverted appended data file") + .isEqualTo( + Sets.newHashSet( + secondSnapshot.manifestListLocation(), // snapshot expired + Iterables.getOnlyElement(secondSnapshotManifests) + .path(), // manifest is no longer referenced + FILE_B.path()) // added, but rolled back + ); checkExpirationResults(1, 0, 0, 1, 1, result); } @@ -1112,10 +1097,9 @@ public void testExpireOlderThanWithDeleteFile() { .map(CharSequence::toString) .collect(Collectors.toSet())); - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", - expectedDeletes, - deletedFiles); + assertThat(deletedFiles) + .as("Should remove expired manifest lists and deleted data file") + .isEqualTo(expectedDeletes); checkExpirationResults(1, 1, 1, 6, 4, result); } @@ -1160,25 +1144,28 @@ public void testExpireAction() { List pending = pendingDeletes.collectAsList(); - Assert.assertEquals( - "Should not change current snapshot", snapshotId, table.currentSnapshot().snapshotId()); - Assert.assertNull( - "Should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - - Assert.assertEquals("Pending deletes should contain one row", 1, pending.size()); - Assert.assertEquals( - "Pending delete should be the expired manifest list location", - firstSnapshot.manifestListLocation(), - pending.get(0).getPath()); - Assert.assertEquals( - "Pending delete should be a manifest list", "Manifest List", pending.get(0).getType()); - - Assert.assertEquals("Should not delete any files", 0, deletedFiles.size()); - - Assert.assertEquals( - "Multiple calls to expire should return the same count of deleted files", - pendingDeletes.count(), - action.expireFiles().count()); + assertThat(table.currentSnapshot().snapshotId()) + .as("Should not change current snapshot.") + .isEqualTo(snapshotId); + + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Should remove the oldest snapshot") + .isNull(); + assertThat(pending).as("Pending deletes should contain one row").hasSize(1); + + assertThat(pending.get(0).getPath()) + .as("Pending delete should be the expired manifest list location") + .isEqualTo(firstSnapshot.manifestListLocation()); + + assertThat(pending.get(0).getType()) + .as("Pending delete should be a manifest list") + .isEqualTo("Manifest List"); + + assertThat(deletedFiles).as("Should not delete any files").hasSize(0); + + assertThat(action.expireFiles().count()) + .as("Multiple calls to expire should return the same count of deleted files") + .isEqualTo(pendingDeletes.count()); } @Test @@ -1208,10 +1195,10 @@ public void testUseLocalIterator() { checkExpirationResults(1L, 0L, 0L, 1L, 2L, results); - Assert.assertEquals( - "Expected total number of jobs with stream-results should match the expected number", - 4L, - jobsRunDuringStreamResults); + assertThat(jobsRunDuringStreamResults) + .as( + "Expected total number of jobs with stream-results should match the expected number") + .isEqualTo(4L); }); } @@ -1244,10 +1231,10 @@ public void testExpireAfterExecute() { checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); List typedExpiredFiles = action.expireFiles().collectAsList(); - Assert.assertEquals("Expired results must match", 1, typedExpiredFiles.size()); + assertThat(typedExpiredFiles).as("Expired results must match").hasSize(1); List untypedExpiredFiles = action.expireFiles().collectAsList(); - Assert.assertEquals("Expired results must match", 1, untypedExpiredFiles.size()); + assertThat(untypedExpiredFiles).as("Expired results must match").hasSize(1); } @Test @@ -1312,8 +1299,9 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe .deleteWith(deletedFiles::add) .execute(); - Assert.assertEquals( - "All reachable files before expiration should be deleted", expectedDeletes, deletedFiles); + assertThat(deletedFiles) + .as("All reachable files before expiration should be deleted") + .isEqualTo(expectedDeletes); } @Test @@ -1344,9 +1332,9 @@ public void testExpireSomeCheckFilesDeleted() { // C, D should be retained (live) // B should be retained (previous snapshot points to it) // A should be deleted - Assert.assertTrue(deletedFiles.contains(FILE_A.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_B.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_C.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_D.path().toString())); + assertThat(deletedFiles).contains(FILE_A.path().toString()); + assertThat(deletedFiles).doesNotContain(FILE_B.path().toString()); + assertThat(deletedFiles).doesNotContain(FILE_C.path().toString()); + assertThat(deletedFiles).doesNotContain(FILE_D.path().toString()); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index a0caacd6d246..6d9f4de6fb57 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -67,7 +69,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkSQLProperties; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.actions.DeleteOrphanFilesSparkAction.StringToFileURI; import org.apache.iceberg.spark.source.FilePathLastModifiedRecord; import org.apache.iceberg.spark.source.ThreeColumnRecord; @@ -78,14 +80,11 @@ import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; -public abstract class TestRemoveOrphanFilesAction extends SparkTestBase { +public abstract class TestRemoveOrphanFilesAction extends TestBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); protected static final Schema SCHEMA = @@ -96,13 +95,13 @@ public abstract class TestRemoveOrphanFilesAction extends SparkTestBase { protected static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).truncate("c2", 2).identity("c3").build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private java.nio.file.Path temp; private File tableDir = null; protected String tableLocation = null; - @Before + @BeforeEach public void setupTableLocation() throws Exception { - this.tableDir = temp.newFolder(); + this.tableDir = temp.resolve("junit").toFile(); this.tableLocation = tableDir.toURI().toString(); } @@ -128,7 +127,7 @@ public void testDryRun() throws IOException, InterruptedException { .select("file_path") .as(Encoders.STRING()) .collectAsList(); - Assert.assertEquals("Should be 2 valid files", 2, validFiles.size()); + assertThat(validFiles).as("Should be 2 valid files").hasSize(2); df.write().mode("append").parquet(tableLocation + "/data"); @@ -139,11 +138,11 @@ public void testDryRun() throws IOException, InterruptedException { .filter(FileStatus::isFile) .map(file -> file.getPath().toString()) .collect(Collectors.toList()); - Assert.assertEquals("Should be 3 files", 3, allFiles.size()); + assertThat(allFiles).as("Should be 3 valid files").hasSize(3); List invalidFiles = Lists.newArrayList(allFiles); invalidFiles.removeAll(validFiles); - Assert.assertEquals("Should be 1 invalid file", 1, invalidFiles.size()); + assertThat(invalidFiles).as("Should be 1 invalid file").hasSize(1); waitUntilAfter(System.currentTimeMillis()); @@ -151,9 +150,9 @@ public void testDryRun() throws IOException, InterruptedException { DeleteOrphanFiles.Result result1 = actions.deleteOrphanFiles(table).deleteWith(s -> {}).execute(); - Assert.assertTrue( - "Default olderThan interval should be safe", - Iterables.isEmpty(result1.orphanFileLocations())); + assertThat(result1.orphanFileLocations()) + .as("Default olderThan interval should be safe") + .isEmpty(); DeleteOrphanFiles.Result result2 = actions @@ -161,14 +160,21 @@ public void testDryRun() throws IOException, InterruptedException { .olderThan(System.currentTimeMillis()) .deleteWith(s -> {}) .execute(); - Assert.assertEquals("Action should find 1 file", invalidFiles, result2.orphanFileLocations()); - Assert.assertTrue("Invalid file should be present", fs.exists(new Path(invalidFiles.get(0)))); + assertThat(result2.orphanFileLocations()) + .as("Action should find 1 file") + .isEqualTo(invalidFiles); + assertThat(fs.exists(new Path(invalidFiles.get(0)))) + .as("Invalid file should be present") + .isTrue(); DeleteOrphanFiles.Result result3 = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals("Action should delete 1 file", invalidFiles, result3.orphanFileLocations()); - Assert.assertFalse( - "Invalid file should not be present", fs.exists(new Path(invalidFiles.get(0)))); + assertThat(result3.orphanFileLocations()) + .as("Action should delete 1 file") + .isEqualTo(invalidFiles); + assertThat(fs.exists(new Path(invalidFiles.get(0)))) + .as("Invalid file should not be present") + .isFalse(); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -177,7 +183,7 @@ public void testDryRun() throws IOException, InterruptedException { Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualRecords).isEqualTo(expectedRecords); } @Test @@ -204,13 +210,13 @@ public void testAllValidFilesAreKept() throws IOException, InterruptedException List snapshots = Lists.newArrayList(table.snapshots()); List snapshotFiles1 = snapshotFiles(snapshots.get(0).snapshotId()); - Assert.assertEquals(1, snapshotFiles1.size()); + assertThat(snapshotFiles1).hasSize(1); List snapshotFiles2 = snapshotFiles(snapshots.get(1).snapshotId()); - Assert.assertEquals(1, snapshotFiles2.size()); + assertThat(snapshotFiles2).hasSize(1); List snapshotFiles3 = snapshotFiles(snapshots.get(2).snapshotId()); - Assert.assertEquals(2, snapshotFiles3.size()); + assertThat(snapshotFiles3).hasSize(2); df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data"); df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data/c2_trunc=AA"); @@ -224,21 +230,21 @@ public void testAllValidFilesAreKept() throws IOException, InterruptedException DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals("Should delete 4 files", 4, Iterables.size(result.orphanFileLocations())); + assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); for (String fileLocation : snapshotFiles1) { - Assert.assertTrue("All snapshot files must remain", fs.exists(new Path(fileLocation))); + assertThat(fs.exists(new Path(fileLocation))).as("All snapshot files must remain").isTrue(); } for (String fileLocation : snapshotFiles2) { - Assert.assertTrue("All snapshot files must remain", fs.exists(new Path(fileLocation))); + assertThat(fs.exists(new Path(fileLocation))).as("All snapshot files must remain").isTrue(); } for (String fileLocation : snapshotFiles3) { - Assert.assertTrue("All snapshot files must remain", fs.exists(new Path(fileLocation))); + assertThat(fs.exists(new Path(fileLocation))).as("All snapshot files must remain").isTrue(); } } @@ -298,12 +304,12 @@ public void orphanedFileRemovedWithParallelTasks() throws InterruptedException, // Verifies that the delete methods ran in the threads created by the provided ExecutorService // ThreadFactory - Assert.assertEquals( - deleteThreads, - Sets.newHashSet( - "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3")); + assertThat(deleteThreads) + .isEqualTo( + Sets.newHashSet( + "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3")); - Assert.assertEquals("Should delete 4 files", 4, deletedFiles.size()); + assertThat(deletedFiles).hasSize(4); } @Test @@ -327,7 +333,9 @@ public void testWapFilesAreKept() throws InterruptedException { Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Should not return data from the staged snapshot", records, actualRecords); + assertThat(actualRecords) + .as("Should not return data from the staged snapshot") + .isEqualTo(records); waitUntilAfter(System.currentTimeMillis()); @@ -336,8 +344,7 @@ public void testWapFilesAreKept() throws InterruptedException { DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertTrue( - "Should not delete any files", Iterables.isEmpty(result.orphanFileLocations())); + assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); } @Test @@ -362,12 +369,12 @@ public void testMetadataFolderIsIntact() throws InterruptedException { DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals("Should delete 1 file", 1, Iterables.size(result.orphanFileLocations())); + assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", records, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(records); } @Test @@ -396,8 +403,7 @@ public void testOlderThanTimestamp() throws InterruptedException { DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(timestamp).execute(); - Assert.assertEquals( - "Should delete only 2 files", 2, Iterables.size(result.orphanFileLocations())); + assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); } @Test @@ -422,11 +428,10 @@ public void testRemoveUnreachableMetadataVersionFiles() throws InterruptedExcept DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals("Should delete 1 file", 1, Iterables.size(result.orphanFileLocations())); - Assert.assertTrue( - "Should remove v1 file", - StreamSupport.stream(result.orphanFileLocations().spliterator(), false) - .anyMatch(file -> file.contains("v1.metadata.json"))); + assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); + assertThat(StreamSupport.stream(result.orphanFileLocations().spliterator(), false)) + .as("Should remove v1 file") + .anyMatch(file -> file.contains("v1.metadata.json")); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -435,7 +440,7 @@ public void testRemoveUnreachableMetadataVersionFiles() throws InterruptedExcept Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); } @Test @@ -458,11 +463,10 @@ public void testManyTopLevelPartitions() throws InterruptedException { DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertTrue( - "Should not delete any files", Iterables.isEmpty(result.orphanFileLocations())); + assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); - Assert.assertEquals("Rows count must match", records.size(), resultDF.count()); + assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); } @Test @@ -485,11 +489,10 @@ public void testManyLeafPartitions() throws InterruptedException { DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertTrue( - "Should not delete any files", Iterables.isEmpty(result.orphanFileLocations())); + assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); - Assert.assertEquals("Row count must match", records.size(), resultDF.count()); + assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); } @Test @@ -522,7 +525,7 @@ public void testHiddenPartitionPaths() throws InterruptedException { DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals("Should delete 2 files", 2, Iterables.size(result.orphanFileLocations())); + assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); } @Test @@ -558,7 +561,7 @@ public void testHiddenPartitionPathsWithPartitionEvolution() throws InterruptedE DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals("Should delete 2 files", 2, Iterables.size(result.orphanFileLocations())); + assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); } @Test @@ -594,8 +597,8 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals("Should delete 0 files", 0, Iterables.size(result.orphanFileLocations())); - Assert.assertTrue(fs.exists(pathToFileInHiddenFolder)); + assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); + assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } private List snapshotFiles(long snapshotId) { @@ -634,7 +637,7 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException, Inte .select("file_path") .as(Encoders.STRING()) .collectAsList(); - Assert.assertEquals("Should be 1 valid files", 1, validFiles.size()); + assertThat(validFiles).as("Should be 1 valid file").hasSize(1); String validFile = validFiles.get(0); df.write().mode("append").parquet(tableLocation + "/data"); @@ -646,11 +649,11 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException, Inte .filter(FileStatus::isFile) .map(file -> file.getPath().toString()) .collect(Collectors.toList()); - Assert.assertEquals("Should be 2 files", 2, allFiles.size()); + assertThat(allFiles).as("Should be 2 files").hasSize(2); List invalidFiles = Lists.newArrayList(allFiles); invalidFiles.removeIf(file -> file.contains(validFile)); - Assert.assertEquals("Should be 1 invalid file", 1, invalidFiles.size()); + assertThat(invalidFiles).as("Should be 1 invalid file").hasSize(1); waitUntilAfter(System.currentTimeMillis()); @@ -661,8 +664,12 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException, Inte .olderThan(System.currentTimeMillis()) .deleteWith(s -> {}) .execute(); - Assert.assertEquals("Action should find 1 file", invalidFiles, result.orphanFileLocations()); - Assert.assertTrue("Invalid file should be present", fs.exists(new Path(invalidFiles.get(0)))); + assertThat(result.orphanFileLocations()) + .as("Action should find 1 file") + .isEqualTo(invalidFiles); + assertThat(fs.exists(new Path(invalidFiles.get(0)))) + .as("Invalid file should be present") + .isTrue(); } @Test @@ -692,13 +699,12 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException DeleteOrphanFiles.Result result = SparkActions.get().deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals( - "Should delete only 1 files", 1, Iterables.size(result.orphanFileLocations())); + assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", records, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(records); } @Test @@ -730,10 +736,9 @@ public void testHiveCatalogTable() throws IOException { .deleteOrphanFiles(table) .olderThan(System.currentTimeMillis() + 1000) .execute(); - Assert.assertTrue( - "trash file should be removed", - StreamSupport.stream(result.orphanFileLocations().spliterator(), false) - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile"))); + assertThat(StreamSupport.stream(result.orphanFileLocations().spliterator(), false)) + .as("trash file should be removed") + .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); } @Test @@ -750,7 +755,7 @@ public void testGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); - Assertions.assertThatThrownBy(() -> SparkActions.get().deleteOrphanFiles(table).execute()) + assertThatThrownBy(() -> SparkActions.get().deleteOrphanFiles(table).execute()) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot delete orphan files: GC is disabled (deleting files may corrupt other tables)"); @@ -781,7 +786,7 @@ public void testCompareToFileList() throws IOException, InterruptedException { file.getPath().toString(), new Timestamp(file.getModificationTime()))) .collect(Collectors.toList()); - Assert.assertEquals("Should be 2 valid files", 2, validFiles.size()); + assertThat(validFiles).as("Should be 2 valid files").hasSize(2); df.write().mode("append").parquet(tableLocation + "/data"); @@ -794,7 +799,7 @@ public void testCompareToFileList() throws IOException, InterruptedException { file.getPath().toString(), new Timestamp(file.getModificationTime()))) .collect(Collectors.toList()); - Assert.assertEquals("Should be 3 files", 3, allFiles.size()); + assertThat(allFiles).as("Should be 3 files").hasSize(3); List invalidFiles = Lists.newArrayList(allFiles); invalidFiles.removeAll(validFiles); @@ -802,7 +807,7 @@ public void testCompareToFileList() throws IOException, InterruptedException { invalidFiles.stream() .map(FilePathLastModifiedRecord::getFilePath) .collect(Collectors.toList()); - Assert.assertEquals("Should be 1 invalid file", 1, invalidFiles.size()); + assertThat(invalidFiles).as("Should be 1 invalid file").hasSize(1); // sleep for 1 second to ensure files will be old enough waitUntilAfter(System.currentTimeMillis()); @@ -821,9 +826,9 @@ public void testCompareToFileList() throws IOException, InterruptedException { .compareToFileList(compareToFileList) .deleteWith(s -> {}) .execute(); - Assert.assertTrue( - "Default olderThan interval should be safe", - Iterables.isEmpty(result1.orphanFileLocations())); + assertThat(result1.orphanFileLocations()) + .as("Default olderThan interval should be safe") + .isEmpty(); DeleteOrphanFiles.Result result2 = actions @@ -832,10 +837,12 @@ public void testCompareToFileList() throws IOException, InterruptedException { .olderThan(System.currentTimeMillis()) .deleteWith(s -> {}) .execute(); - Assert.assertEquals( - "Action should find 1 file", invalidFilePaths, result2.orphanFileLocations()); - Assert.assertTrue( - "Invalid file should be present", fs.exists(new Path(invalidFilePaths.get(0)))); + assertThat(result2.orphanFileLocations()) + .as("Action should find 1 file") + .isEqualTo(invalidFilePaths); + assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) + .as("Invalid file should be present") + .isTrue(); DeleteOrphanFiles.Result result3 = actions @@ -843,10 +850,12 @@ public void testCompareToFileList() throws IOException, InterruptedException { .compareToFileList(compareToFileList) .olderThan(System.currentTimeMillis()) .execute(); - Assert.assertEquals( - "Action should delete 1 file", invalidFilePaths, result3.orphanFileLocations()); - Assert.assertFalse( - "Invalid file should not be present", fs.exists(new Path(invalidFilePaths.get(0)))); + assertThat(result3.orphanFileLocations()) + .as("Action should delete 1 file") + .isEqualTo(invalidFilePaths); + assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) + .as("Invalid file should not be present") + .isFalse(); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -855,7 +864,7 @@ public void testCompareToFileList() throws IOException, InterruptedException { Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); List outsideLocationMockFiles = Lists.newArrayList(new FilePathLastModifiedRecord("/tmp/mock1", new Timestamp(0L))); @@ -872,8 +881,7 @@ public void testCompareToFileList() throws IOException, InterruptedException { .compareToFileList(compareToFileListWithOutsideLocation) .deleteWith(s -> {}) .execute(); - Assert.assertEquals( - "Action should find nothing", Lists.newArrayList(), result4.orphanFileLocations()); + assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); } protected long waitUntilAfter(long timestampMillis) { @@ -938,8 +946,8 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .olderThan(System.currentTimeMillis() + 1000) .execute(); - Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); - Assertions.assertThat(statsLocation.length()) + assertThat(statsLocation).as("stats file should exist").exists(); + assertThat(statsLocation.length()) .as("stats file length") .isEqualTo(statisticsFile.fileSizeInBytes()); @@ -953,11 +961,11 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .olderThan(System.currentTimeMillis() + 1000) .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); - Assertions.assertThat(orphanFileLocations).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(orphanFileLocations)) + assertThat(orphanFileLocations).as("Should be orphan file").hasSize(1); + assertThat(Iterables.getOnlyElement(orphanFileLocations)) .as("Deleted file") .isEqualTo(statsLocation.toURI().toString()); - Assertions.assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); + assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); } @Test @@ -985,7 +993,7 @@ public void testPathsWithActualFileHavingNoAuthority() { public void testPathsWithEqualSchemes() { List validFiles = Lists.newArrayList("scheme1://bucket1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("scheme2://bucket1/dir1/dir2/file1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> executeTest( validFiles, @@ -1014,7 +1022,7 @@ public void testPathsWithEqualSchemes() { public void testPathsWithEqualAuthorities() { List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> executeTest( validFiles, @@ -1080,6 +1088,6 @@ private void executeTest( List orphanFiles = DeleteOrphanFilesSparkAction.findOrphanFiles( spark, toFileUri.apply(actualFileDS), toFileUri.apply(validFileDS), mode); - Assert.assertEquals(expectedOrphanFiles, orphanFiles); + assertThat(orphanFiles).isEqualTo(expectedOrphanFiles); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 0abfd79d5ddb..2476d1bb7078 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.actions; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.util.Map; import java.util.stream.StreamSupport; @@ -29,9 +31,8 @@ import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.expressions.Transform; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; public class TestRemoveOrphanFilesAction3 extends TestRemoveOrphanFilesAction { @Test @@ -58,10 +59,9 @@ public void testSparkCatalogTable() throws Exception { .deleteOrphanFiles(table.table()) .olderThan(System.currentTimeMillis() + 1000) .execute(); - Assert.assertTrue( - "trash file should be removed", - StreamSupport.stream(results.orphanFileLocations().spliterator(), false) - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile"))); + assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) + .as("trash file should be removed") + .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); } @Test @@ -88,10 +88,9 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { .deleteOrphanFiles(table.table()) .olderThan(System.currentTimeMillis() + 1000) .execute(); - Assert.assertTrue( - "trash file should be removed", - StreamSupport.stream(results.orphanFileLocations().spliterator(), false) - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile"))); + assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) + .as("trash file should be removed") + .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); } @Test @@ -118,10 +117,10 @@ public void testSparkCatalogNamedHiveTable() throws Exception { .deleteOrphanFiles(table.table()) .olderThan(System.currentTimeMillis() + 1000) .execute(); - Assert.assertTrue( - "trash file should be removed", - StreamSupport.stream(results.orphanFileLocations().spliterator(), false) - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile"))); + + assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) + .as("trash file should be removed") + .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); } @Test @@ -151,10 +150,9 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { .deleteOrphanFiles(table.table()) .olderThan(System.currentTimeMillis() + 1000) .execute(); - Assert.assertTrue( - "trash file should be removed", - StreamSupport.stream(results.orphanFileLocations().spliterator(), false) - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile"))); + assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) + .as("trash file should be removed") + .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); } @Test @@ -184,13 +182,12 @@ public void testSparkSessionCatalogHiveTable() throws Exception { .deleteOrphanFiles(table.table()) .olderThan(System.currentTimeMillis() + 1000) .execute(); - Assert.assertTrue( - "trash file should be removed", - StreamSupport.stream(results.orphanFileLocations().spliterator(), false) - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile"))); + assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) + .as("trash file should be removed") + .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); } - @After + @AfterEach public void resetSparkSessionCatalog() throws Exception { spark.conf().unset("spark.sql.catalog.spark_catalog"); spark.conf().unset("spark.sql.catalog.spark_catalog.type"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index bfffa65accac..d0d22e46ffc4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -25,6 +25,7 @@ import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.doAnswer; @@ -36,6 +37,7 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.file.Path; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -82,7 +84,6 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -92,8 +93,8 @@ import org.apache.iceberg.spark.FileRewriteCoordinator; import org.apache.iceberg.spark.ScanTaskSetManager; import org.apache.iceberg.spark.SparkTableUtil; -import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.RewriteExecutionContext; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.ThreeColumnRecord; @@ -106,17 +107,14 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import org.mockito.ArgumentMatcher; import org.mockito.Mockito; -public class TestRewriteDataFilesAction extends SparkTestBase { +public class TestRewriteDataFilesAction extends TestBase { private static final int SCALE = 400000; @@ -127,21 +125,21 @@ public class TestRewriteDataFilesAction extends SparkTestBase { optional(2, "c2", Types.StringType.get()), optional(3, "c3", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); private final ScanTaskSetManager manager = ScanTaskSetManager.get(); private String tableLocation = null; - @BeforeClass + @BeforeAll public static void setupSpark() { // disable AQE as tests assume that writes generate a particular number of files spark.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false"); } - @Before + @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = temp.newFolder(); + File tableDir = temp.resolve("junit").toFile(); this.tableLocation = tableDir.toURI().toString(); } @@ -157,11 +155,11 @@ public void testEmptyTable() { Map options = Maps.newHashMap(); Table table = TABLES.create(SCHEMA, spec, options, tableLocation); - Assert.assertNull("Table must be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); basicRewrite(table).execute(); - Assert.assertNull("Table must stay empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).as("Table must stay empty").isNull(); } @Test @@ -172,8 +170,10 @@ public void testBinPackUnpartitionedTable() { long dataSizeBefore = testDataSize(table); Result result = basicRewrite(table).execute(); - Assert.assertEquals("Action should rewrite 4 data files", 4, result.rewrittenDataFilesCount()); - Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 4 data files") + .isEqualTo(4); + assertThat(result.addedDataFilesCount()).as("Action should add 1 data file").isOne(); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); shouldHaveFiles(table, 1); @@ -190,8 +190,10 @@ public void testBinPackPartitionedTable() { long dataSizeBefore = testDataSize(table); Result result = basicRewrite(table).execute(); - Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount()); - Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 8 data files") + .isEqualTo(8); + assertThat(result.addedDataFilesCount()).as("Action should add 4 data file").isEqualTo(4); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); shouldHaveFiles(table, 4); @@ -213,8 +215,10 @@ public void testBinPackWithFilter() { .filter(Expressions.startsWith("c2", "foo")) .execute(); - Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount()); - Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 2 data files") + .isEqualTo(2); + assertThat(result.addedDataFilesCount()).as("Action should add 1 data file").isOne(); assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); shouldHaveFiles(table, 7); @@ -271,10 +275,9 @@ public void testBinPackAfterPartitionChange() { Integer.toString(averageFileSize(table) + 1001)) .execute(); - Assert.assertEquals( - "Should have 1 fileGroup because all files were not correctly partitioned", - 1, - result.rewriteResults().size()); + assertThat(result.rewriteResults()) + .as("Should have 1 fileGroup because all files were not correctly partitioned") + .hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); List postRewriteData = currentData(); @@ -320,12 +323,14 @@ public void testBinPackWithDeletes() { .option(SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) .option(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "2") .execute(); - Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 2 data files") + .isEqualTo(2); assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); List actualRecords = currentData(); assertEquals("Rows must match", expectedRecords, actualRecords); - Assert.assertEquals("7 rows are removed", total - 7, actualRecords.size()); + assertThat(actualRecords).as("7 rows are removed").hasSize(total - 7); } @Test @@ -353,23 +358,22 @@ public void testBinPackWithDeleteAllData() { .rewriteDataFiles(table) .option(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "1") .execute(); - Assert.assertEquals("Action should rewrite 1 data files", 1, result.rewrittenDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()).as("Action should rewrite 1 data files").isOne(); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); List actualRecords = currentData(); assertEquals("Rows must match", expectedRecords, actualRecords); - Assert.assertEquals( - "Data manifest should not have existing data file", - 0, - (long) table.currentSnapshot().dataManifests(table.io()).get(0).existingFilesCount()); - Assert.assertEquals( - "Data manifest should have 1 delete data file", - 1L, - (long) table.currentSnapshot().dataManifests(table.io()).get(0).deletedFilesCount()); - Assert.assertEquals( - "Delete manifest added row count should equal total count", - total, - (long) table.currentSnapshot().deleteManifests(table.io()).get(0).addedRowsCount()); + assertThat(table.currentSnapshot().dataManifests(table.io()).get(0).existingFilesCount()) + .as("Data manifest should not have existing data file") + .isZero(); + + assertThat((long) table.currentSnapshot().dataManifests(table.io()).get(0).deletedFilesCount()) + .as("Data manifest should have 1 delete data file") + .isEqualTo(1L); + + assertThat(table.currentSnapshot().deleteManifests(table.io()).get(0).addedRowsCount()) + .as("Delete manifest added row count should equal total count") + .isEqualTo(total); } @Test @@ -384,8 +388,10 @@ public void testBinPackWithStartingSequenceNumber() { Result result = basicRewrite(table).option(RewriteDataFiles.USE_STARTING_SEQUENCE_NUMBER, "true").execute(); - Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount()); - Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 8 data files") + .isEqualTo(8); + assertThat(result.addedDataFilesCount()).as("Action should add 4 data files").isEqualTo(4); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); shouldHaveFiles(table, 4); @@ -393,15 +399,16 @@ public void testBinPackWithStartingSequenceNumber() { assertEquals("Rows must match", expectedRecords, actualRecords); table.refresh(); - Assert.assertTrue( - "Table sequence number should be incremented", - oldSequenceNumber < table.currentSnapshot().sequenceNumber()); + assertThat(table.currentSnapshot().sequenceNumber()) + .as("Table sequence number should be incremented") + .isGreaterThan(oldSequenceNumber); Dataset rows = SparkTableUtil.loadMetadataTable(spark, table, MetadataTableType.ENTRIES); for (Row row : rows.collectAsList()) { if (row.getInt(0) == 1) { - Assert.assertEquals( - "Expect old sequence number for added entries", oldSequenceNumber, row.getLong(2)); + assertThat(row.getLong(2)) + .as("Expect old sequence number for added entries") + .isEqualTo(oldSequenceNumber); } } } @@ -414,13 +421,15 @@ public void testBinPackWithStartingSequenceNumberV1Compatibility() { List expectedRecords = currentData(); table.refresh(); long oldSequenceNumber = table.currentSnapshot().sequenceNumber(); - Assert.assertEquals("Table sequence number should be 0", 0, oldSequenceNumber); + assertThat(oldSequenceNumber).as("Table sequence number should be 0").isZero(); long dataSizeBefore = testDataSize(table); Result result = basicRewrite(table).option(RewriteDataFiles.USE_STARTING_SEQUENCE_NUMBER, "true").execute(); - Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount()); - Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 8 data files") + .isEqualTo(8); + assertThat(result.addedDataFilesCount()).as("Action should add 4 data files").isEqualTo(4); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); shouldHaveFiles(table, 4); @@ -428,15 +437,15 @@ public void testBinPackWithStartingSequenceNumberV1Compatibility() { assertEquals("Rows must match", expectedRecords, actualRecords); table.refresh(); - Assert.assertEquals( - "Table sequence number should still be 0", - oldSequenceNumber, - table.currentSnapshot().sequenceNumber()); + assertThat(table.currentSnapshot().sequenceNumber()) + .as("Table sequence number should still be 0") + .isEqualTo(oldSequenceNumber); Dataset rows = SparkTableUtil.loadMetadataTable(spark, table, MetadataTableType.ENTRIES); for (Row row : rows.collectAsList()) { - Assert.assertEquals( - "Expect sequence number 0 for all entries", oldSequenceNumber, row.getLong(2)); + assertThat(row.getLong(2)) + .as("Expect sequence number 0 for all entries") + .isEqualTo(oldSequenceNumber); } } @@ -462,15 +471,19 @@ public void testRewriteLargeTableHasResiduals() { CloseableIterable tasks = table.newScan().ignoreResiduals().filter(Expressions.equal("c3", "0")).planFiles(); for (FileScanTask task : tasks) { - Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual()); + assertThat(task.residual()) + .as("Residuals must be ignored") + .isEqualTo(Expressions.alwaysTrue()); } shouldHaveFiles(table, 2); long dataSizeBefore = testDataSize(table); Result result = basicRewrite(table).filter(Expressions.equal("c3", "0")).execute(); - Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount()); - Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 2 data files") + .isEqualTo(2); + assertThat(result.addedDataFilesCount()).as("Action should add 1 data file").isOne(); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); List actualRecords = currentData(); @@ -493,8 +506,8 @@ public void testBinPackSplitLargeFile() { .option(SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, Long.toString(targetSize * 2 - 2000)) .execute(); - Assert.assertEquals("Action should delete 1 data files", 1, result.rewrittenDataFilesCount()); - Assert.assertEquals("Action should add 2 data files", 2, result.addedDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()).as("Action should delete 1 data files").isOne(); + assertThat(result.addedDataFilesCount()).as("Action should add 2 data files").isEqualTo(2); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); shouldHaveFiles(table, 2); @@ -525,10 +538,12 @@ public void testBinPackCombineMixedFiles() { .option(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 1000)) .execute(); - Assert.assertEquals("Action should delete 3 data files", 3, result.rewrittenDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should delete 3 data files") + .isEqualTo(3); // Should Split the big files into 3 pieces, one of which should be combined with the two // smaller files - Assert.assertEquals("Action should add 3 data files", 3, result.addedDataFilesCount()); + assertThat(result.addedDataFilesCount()).as("Action should add 3 data files").isEqualTo(3); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); shouldHaveFiles(table, 3); @@ -558,8 +573,10 @@ public void testBinPackCombineMediumFiles() { Integer.toString(targetSize - 100)) // All files too small .execute(); - Assert.assertEquals("Action should delete 4 data files", 4, result.rewrittenDataFilesCount()); - Assert.assertEquals("Action should add 3 data files", 3, result.addedDataFilesCount()); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should delete 4 data files") + .isEqualTo(4); + assertThat(result.addedDataFilesCount()).as("Action should add 3 data files").isEqualTo(3); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); shouldHaveFiles(table, 3); @@ -587,7 +604,7 @@ public void testPartialProgressEnabled() { .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "10") .execute(); - Assert.assertEquals("Should have 10 fileGroups", result.rewriteResults().size(), 10); + assertThat(result.rewriteResults()).as("Should have 10 fileGroups").hasSize(10); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); table.refresh(); @@ -615,7 +632,7 @@ public void testMultipleGroups() { .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") .execute(); - Assert.assertEquals("Should have 10 fileGroups", result.rewriteResults().size(), 10); + assertThat(result.rewriteResults()).as("Should have 10 fileGroups").hasSize(10); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); table.refresh(); @@ -644,7 +661,7 @@ public void testPartialProgressMaxCommits() { .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3") .execute(); - Assert.assertEquals("Should have 10 fileGroups", result.rewriteResults().size(), 10); + assertThat(result.rewriteResults()).as("Should have 10 fileGroups").hasSize(10); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); table.refresh(); @@ -676,7 +693,7 @@ public void testSingleCommitWithRewriteFailure() { .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); - Assertions.assertThatThrownBy(spyRewrite::execute) + assertThatThrownBy(spyRewrite::execute) .isInstanceOf(RuntimeException.class) .hasMessage("Rewrite Failed"); @@ -710,7 +727,7 @@ public void testSingleCommitWithCommitFailure() { doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); - Assertions.assertThatThrownBy(spyRewrite::execute) + assertThatThrownBy(spyRewrite::execute) .isInstanceOf(RuntimeException.class) .hasMessage("Commit Failure"); @@ -745,7 +762,7 @@ public void testParallelSingleCommitWithRewriteFailure() { .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); - Assertions.assertThatThrownBy(spyRewrite::execute) + assertThatThrownBy(spyRewrite::execute) .isInstanceOf(RuntimeException.class) .hasMessage("Rewrite Failed"); @@ -875,7 +892,7 @@ public void testParallelPartialProgressWithCommitFailure() { RewriteDataFiles.Result result = spyRewrite.execute(); // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups comitted - Assert.assertEquals("Should have 6 fileGroups", 6, result.rewriteResults().size()); + assertThat(result.rewriteResults()).as("Should have 6 fileGroups").hasSize(6); assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); table.refresh(); @@ -893,7 +910,7 @@ public void testParallelPartialProgressWithCommitFailure() { public void testInvalidOptions() { Table table = createTable(20); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table) .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") @@ -904,7 +921,7 @@ public void testInvalidOptions() { "Cannot set partial-progress.max-commits to -5, " + "the value must be positive when partial-progress.enabled is true"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table) .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5") @@ -913,17 +930,17 @@ public void testInvalidOptions() { .hasMessage( "Cannot set max-concurrent-file-group-rewrites to -5, the value must be positive."); - Assertions.assertThatThrownBy(() -> basicRewrite(table).option("foobarity", "-5").execute()) + assertThatThrownBy(() -> basicRewrite(table).option("foobarity", "-5").execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot use options [foobarity], they are not supported by the action or the rewriter BIN-PACK"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table).option(RewriteDataFiles.REWRITE_JOB_ORDER, "foo").execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid rewrite job order name: foo"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table) .sort(SortOrder.builderFor(table.schema()).asc("c2").build()) @@ -953,7 +970,7 @@ public void testSortMultipleGroups() { RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) .execute(); - Assert.assertEquals("Should have 10 fileGroups", result.rewriteResults().size(), 10); + assertThat(result.rewriteResults()).as("Should have 10 fileGroups").hasSize(10); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); table.refresh(); @@ -984,7 +1001,7 @@ public void testSimpleSort() { RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table))) .execute(); - Assert.assertEquals("Should have 1 fileGroups", result.rewriteResults().size(), 1); + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); table.refresh(); @@ -1018,10 +1035,9 @@ public void testSortAfterPartitionChange() { RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table))) .execute(); - Assert.assertEquals( - "Should have 1 fileGroup because all files were not correctly partitioned", - result.rewriteResults().size(), - 1); + assertThat(result.rewriteResults()) + .as("Should have 1 fileGroups because all files were not correctly partitioned") + .hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); table.refresh(); @@ -1052,7 +1068,7 @@ public void testSortCustomSortOrder() { RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table))) .execute(); - Assert.assertEquals("Should have 1 fileGroups", result.rewriteResults().size(), 1); + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); table.refresh(); @@ -1091,7 +1107,7 @@ public void testSortCustomSortOrderRequiresRepartition() { Integer.toString(averageFileSize(table) / partitions)) .execute(); - Assert.assertEquals("Should have 1 fileGroups", result.rewriteResults().size(), 1); + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); table.refresh(); @@ -1128,11 +1144,12 @@ public void testAutoSortShuffleOutput() { .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") .execute(); - Assert.assertEquals("Should have 1 fileGroups", result.rewriteResults().size(), 1); + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - Assert.assertTrue( - "Should have written 40+ files", - Iterables.size(table.currentSnapshot().addedDataFiles(table.io())) >= 40); + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(table.currentSnapshot().addedDataFiles(table.io())) + .as("Should have written 40+ files") + .hasSizeGreaterThanOrEqualTo(40); table.refresh(); @@ -1166,7 +1183,7 @@ public void testCommitStateUnknownException() { doReturn(util).when(spyAction).commitManager(table.currentSnapshot().snapshotId()); - Assertions.assertThatThrownBy(spyAction::execute) + assertThatThrownBy(spyAction::execute) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith( "Unknown State\n" + "Cannot determine whether the commit was successful or not"); @@ -1190,8 +1207,8 @@ public void testZOrderSort() { double originalFilesC2C3 = percentFilesRequired(table, new String[] {"c2", "c3"}, new String[] {"foo23", "bar23"}); - Assert.assertTrue("Should require all files to scan c2", originalFilesC2 > 0.99); - Assert.assertTrue("Should require all files to scan c3", originalFilesC3 > 0.99); + assertThat(originalFilesC2).as("Should require all files to scan c2").isGreaterThan(0.99); + assertThat(originalFilesC3).as("Should require all files to scan c3").isGreaterThan(0.99); long dataSizeBefore = testDataSize(table); RewriteDataFiles.Result result = @@ -1207,10 +1224,11 @@ public void testZOrderSort() { .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") .execute(); - Assert.assertEquals("Should have 1 fileGroups", 1, result.rewriteResults().size()); + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - int zOrderedFilesTotal = Iterables.size(table.currentSnapshot().addedDataFiles(table.io())); - Assert.assertTrue("Should have written 40+ files", zOrderedFilesTotal >= 40); + assertThat(table.currentSnapshot().addedDataFiles(table.io())) + .as("Should have written 40+ files") + .hasSizeGreaterThanOrEqualTo(40); table.refresh(); @@ -1225,15 +1243,15 @@ public void testZOrderSort() { double filesScannedC2C3 = percentFilesRequired(table, new String[] {"c2", "c3"}, new String[] {"foo23", "bar23"}); - Assert.assertTrue( - "Should have reduced the number of files required for c2", - filesScannedC2 < originalFilesC2); - Assert.assertTrue( - "Should have reduced the number of files required for c3", - filesScannedC3 < originalFilesC3); - Assert.assertTrue( - "Should have reduced the number of files required for a c2,c3 predicate", - filesScannedC2C3 < originalFilesC2C3); + assertThat(originalFilesC2) + .as("Should have reduced the number of files required for c2") + .isGreaterThan(filesScannedC2); + assertThat(originalFilesC3) + .as("Should have reduced the number of files required for c3") + .isGreaterThan(filesScannedC3); + assertThat(originalFilesC2C3) + .as("Should have reduced the number of files required for c2,c3 predicate") + .isGreaterThan(filesScannedC2C3); } @Test @@ -1263,10 +1281,11 @@ public void testZOrderAllTypesSort() { .option(SizeBasedFileRewriter.REWRITE_ALL, "true") .execute(); - Assert.assertEquals("Should have 1 fileGroups", 1, result.rewriteResults().size()); + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - int zOrderedFilesTotal = Iterables.size(table.currentSnapshot().addedDataFiles(table.io())); - Assert.assertEquals("Should have written 1 file", 1, zOrderedFilesTotal); + assertThat(table.currentSnapshot().addedDataFiles(table.io())) + .as("Should have written 1 file") + .hasSize(1); table.refresh(); @@ -1285,15 +1304,15 @@ public void testInvalidAPIUsage() { SortOrder sortOrder = SortOrder.builderFor(table.schema()).asc("c2").build(); - Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).binPack().sort()) + assertThatThrownBy(() -> actions().rewriteDataFiles(table).binPack().sort()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); - Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); - Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); } @@ -1325,9 +1344,9 @@ public void testRewriteJobOrderBytesAsc() { .collect(Collectors.toList()); expected.sort(Comparator.naturalOrder()); - Assert.assertEquals("Size in bytes order should be ascending", actual, expected); + assertThat(actual).as("Size in bytes order should be ascending").isEqualTo(expected); Collections.reverse(expected); - Assert.assertNotEquals("Size in bytes order should not be descending", actual, expected); + assertThat(actual).as("Size in bytes order should not be descending").isNotEqualTo(expected); } @Test @@ -1357,9 +1376,9 @@ public void testRewriteJobOrderBytesDesc() { .collect(Collectors.toList()); expected.sort(Comparator.reverseOrder()); - Assert.assertEquals("Size in bytes order should be descending", actual, expected); + assertThat(actual).as("Size in bytes order should be descending").isEqualTo(expected); Collections.reverse(expected); - Assert.assertNotEquals("Size in bytes order should not be ascending", actual, expected); + assertThat(actual).as("Size in bytes order should not be ascending").isNotEqualTo(expected); } @Test @@ -1389,9 +1408,9 @@ public void testRewriteJobOrderFilesAsc() { .collect(Collectors.toList()); expected.sort(Comparator.naturalOrder()); - Assert.assertEquals("Number of files order should be ascending", actual, expected); + assertThat(actual).as("Number of files order should be ascending").isEqualTo(expected); Collections.reverse(expected); - Assert.assertNotEquals("Number of files order should not be descending", actual, expected); + assertThat(actual).as("Number of files order should not be descending").isNotEqualTo(expected); } @Test @@ -1421,9 +1440,9 @@ public void testRewriteJobOrderFilesDesc() { .collect(Collectors.toList()); expected.sort(Comparator.reverseOrder()); - Assert.assertEquals("Number of files order should be descending", actual, expected); + assertThat(actual).as("Number of files order should be descending").isEqualTo(expected); Collections.reverse(expected); - Assert.assertNotEquals("Number of files order should not be ascending", actual, expected); + assertThat(actual).as("Number of files order should not be ascending").isNotEqualTo(expected); } private Stream toGroupStream(Table table, RewriteDataFilesSparkAction rewrite) { @@ -1447,48 +1466,50 @@ protected long testDataSize(Table table) { protected void shouldHaveMultipleFiles(Table table) { table.refresh(); int numFiles = Iterables.size(table.newScan().planFiles()); - Assert.assertTrue(String.format("Should have multiple files, had %d", numFiles), numFiles > 1); + assertThat(numFiles) + .as(String.format("Should have multiple files, had %d", numFiles)) + .isGreaterThan(1); } protected void shouldHaveFiles(Table table, int numExpected) { table.refresh(); int numFiles = Iterables.size(table.newScan().planFiles()); - Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles); + assertThat(numFiles).as("Did not have the expected number of files").isEqualTo(numExpected); } protected void shouldHaveSnapshots(Table table, int expectedSnapshots) { table.refresh(); int actualSnapshots = Iterables.size(table.snapshots()); - Assert.assertEquals( - "Table did not have the expected number of snapshots", expectedSnapshots, actualSnapshots); + assertThat(actualSnapshots) + .as("Table did not have the expected number of snapshots") + .isEqualTo(expectedSnapshots); } protected void shouldHaveNoOrphans(Table table) { - Assert.assertEquals( - "Should not have found any orphan files", - ImmutableList.of(), - actions() - .deleteOrphanFiles(table) - .olderThan(System.currentTimeMillis()) - .execute() - .orphanFileLocations()); + assertThat( + actions() + .deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis()) + .execute() + .orphanFileLocations()) + .as("Should not have found any orphan files") + .isEmpty(); } protected void shouldHaveACleanCache(Table table) { - Assert.assertEquals( - "Should not have any entries in cache", ImmutableSet.of(), cacheContents(table)); + assertThat(cacheContents(table)).as("Should not have any entries in cache").isEmpty(); } protected void shouldHaveLastCommitSorted(Table table, String column) { List, Pair>> overlappingFiles = checkForOverlappingFiles(table, column); - Assert.assertEquals("Found overlapping files", Collections.emptyList(), overlappingFiles); + assertThat(overlappingFiles).as("Found overlapping files").isEmpty(); } protected void shouldHaveLastCommitUnsorted(Table table, String column) { List, Pair>> overlappingFiles = checkForOverlappingFiles(table, column); - Assert.assertNotEquals("Found no overlapping files", Collections.emptyList(), overlappingFiles); + assertThat(overlappingFiles).as("Found no overlapping files").isNotEmpty(); } private Pair boundsOf(DataFile file, NestedField field, Class javaClass) { @@ -1567,7 +1588,7 @@ protected Table createTable() { .updateProperties() .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, Integer.toString(20 * 1024)) .commit(); - Assert.assertNull("Table must be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); return table; } @@ -1587,7 +1608,7 @@ protected Table createTablePartitioned( int partitions, int files, int numRecords, Map options) { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 2).build(); Table table = TABLES.create(SCHEMA, spec, options, tableLocation); - Assert.assertNull("Table must be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); writeRecords(files, numRecords, partitions); return table; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index e7326c73e838..a449de414a10 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -25,6 +25,7 @@ import static org.apache.iceberg.ValidationHelpers.validateDataManifest; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; @@ -32,6 +33,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.UUID; @@ -45,6 +47,9 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; @@ -65,8 +70,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.SparkTableUtil; -import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.source.ThreeColumnRecord; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; @@ -75,18 +80,13 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.TableIdentifier; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class TestRewriteManifestsAction extends SparkTestBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteManifestsAction extends TestBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); private static final Schema SCHEMA = @@ -95,39 +95,40 @@ public class TestRewriteManifestsAction extends SparkTestBase { optional(2, "c2", Types.StringType.get()), optional(3, "c3", Types.StringType.get())); - @Parameters(name = "snapshotIdInheritanceEnabled = {0}, useCaching = {1}, formatVersion = {2}") + @Parameters( + name = + "snapshotIdInheritanceEnabled = {0}, useCaching = {1}, shouldStageManifests = {2}, formatVersion = {3}") public static Object[] parameters() { return new Object[][] { - new Object[] {"true", "true", 1}, - new Object[] {"false", "true", 1}, - new Object[] {"true", "false", 2}, - new Object[] {"false", "false", 2} + new Object[] {"true", "true", false, 1}, + new Object[] {"false", "true", true, 1}, + new Object[] {"true", "false", false, 2}, + new Object[] {"false", "false", false, 2} }; } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @Parameter private String snapshotIdInheritanceEnabled; + + @Parameter(index = 1) + private String useCaching; + + @Parameter(index = 2) + private boolean shouldStageManifests; + + @Parameter(index = 3) + private int formatVersion; - private final String snapshotIdInheritanceEnabled; - private final String useCaching; - private final int formatVersion; - private final boolean shouldStageManifests; private String tableLocation = null; - public TestRewriteManifestsAction( - String snapshotIdInheritanceEnabled, String useCaching, int formatVersion) { - this.snapshotIdInheritanceEnabled = snapshotIdInheritanceEnabled; - this.useCaching = useCaching; - this.formatVersion = formatVersion; - this.shouldStageManifests = formatVersion == 1 && snapshotIdInheritanceEnabled.equals("false"); - } + @TempDir private Path temp; - @Before + @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = temp.newFolder(); + File tableDir = temp.resolve("junit").toFile(); this.tableLocation = tableDir.toURI().toString(); } - @Test + @TestTemplate public void testRewriteManifestsEmptyTable() throws IOException { PartitionSpec spec = PartitionSpec.unpartitioned(); Map options = Maps.newHashMap(); @@ -135,7 +136,7 @@ public void testRewriteManifestsEmptyTable() throws IOException { options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); Table table = TABLES.create(SCHEMA, spec, options, tableLocation); - Assert.assertNull("Table must be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); SparkActions actions = SparkActions.get(); @@ -143,13 +144,13 @@ public void testRewriteManifestsEmptyTable() throws IOException { .rewriteManifests(table) .rewriteIf(manifest -> true) .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) - .stagingLocation(temp.newFolder().toString()) + .stagingLocation(java.nio.file.Files.createTempDirectory(temp, "junit").toString()) .execute(); - Assert.assertNull("Table must stay empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).as("Table must stay empty").isNull(); } - @Test + @TestTemplate public void testRewriteSmallManifestsNonPartitionedTable() { PartitionSpec spec = PartitionSpec.unpartitioned(); Map options = Maps.newHashMap(); @@ -171,7 +172,7 @@ public void testRewriteSmallManifestsNonPartitionedTable() { table.refresh(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 2 manifests before rewrite", 2, manifests.size()); + assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); SparkActions actions = SparkActions.get(); @@ -182,20 +183,18 @@ public void testRewriteSmallManifestsNonPartitionedTable() { .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) .execute(); - Assert.assertEquals( - "Action should rewrite 2 manifests", 2, Iterables.size(result.rewrittenManifests())); - Assert.assertEquals( - "Action should add 1 manifests", 1, Iterables.size(result.addedManifests())); + assertThat(result.rewrittenManifests()).as("Action should rewrite 2 manifests").hasSize(2); + assertThat(result.addedManifests()).as("Action should add 1 manifests").hasSize(1); assertManifestsLocation(result.addedManifests()); table.refresh(); List newManifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 1 manifests after rewrite", 1, newManifests.size()); + assertThat(newManifests).as("Should have 1 manifests after rewrite").hasSize(1); - Assert.assertEquals(4, (long) newManifests.get(0).existingFilesCount()); - Assert.assertFalse(newManifests.get(0).hasAddedFiles()); - Assert.assertFalse(newManifests.get(0).hasDeletedFiles()); + assertThat(newManifests.get(0).existingFilesCount()).isEqualTo(4); + assertThat(newManifests.get(0).hasAddedFiles()).isFalse(); + assertThat(newManifests.get(0).hasDeletedFiles()).isFalse(); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records1); @@ -205,10 +204,10 @@ public void testRewriteSmallManifestsNonPartitionedTable() { List actualRecords = resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); } - @Test + @TestTemplate public void testRewriteManifestsWithCommitStateUnknownException() { PartitionSpec spec = PartitionSpec.unpartitioned(); Map options = Maps.newHashMap(); @@ -230,7 +229,7 @@ public void testRewriteManifestsWithCommitStateUnknownException() { table.refresh(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 2 manifests before rewrite", 2, manifests.size()); + assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); SparkActions actions = SparkActions.get(); @@ -248,7 +247,7 @@ public void testRewriteManifestsWithCommitStateUnknownException() { Table spyTable = spy(table); when(spyTable.rewriteManifests()).thenReturn(spyNewRewriteManifests); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> actions.rewriteManifests(spyTable).rewriteIf(manifest -> true).execute()) .cause() .isInstanceOf(RuntimeException.class) @@ -258,11 +257,11 @@ public void testRewriteManifestsWithCommitStateUnknownException() { // table should reflect the changes, since the commit was successful List newManifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 1 manifests after rewrite", 1, newManifests.size()); + assertThat(newManifests).as("Should have 1 manifests after rewrite").hasSize(1); - Assert.assertEquals(4, (long) newManifests.get(0).existingFilesCount()); - Assert.assertFalse(newManifests.get(0).hasAddedFiles()); - Assert.assertFalse(newManifests.get(0).hasDeletedFiles()); + assertThat(newManifests.get(0).existingFilesCount()).isEqualTo(4); + assertThat(newManifests.get(0).hasAddedFiles()).isFalse(); + assertThat(newManifests.get(0).hasDeletedFiles()).isFalse(); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records1); @@ -272,10 +271,10 @@ public void testRewriteManifestsWithCommitStateUnknownException() { List actualRecords = resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); } - @Test + @TestTemplate public void testRewriteSmallManifestsPartitionedTable() { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 2).build(); Map options = Maps.newHashMap(); @@ -309,7 +308,7 @@ public void testRewriteSmallManifestsPartitionedTable() { table.refresh(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 4 manifests before rewrite", 4, manifests.size()); + assertThat(manifests).as("Should have 4 manifests before rewrite").hasSize(4); SparkActions actions = SparkActions.get(); @@ -329,24 +328,23 @@ public void testRewriteSmallManifestsPartitionedTable() { .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) .execute(); - Assert.assertEquals( - "Action should rewrite 4 manifests", 4, Iterables.size(result.rewrittenManifests())); - Assert.assertEquals( - "Action should add 2 manifests", 2, Iterables.size(result.addedManifests())); + assertThat(result.rewrittenManifests()).as("Action should rewrite 4 manifests").hasSize(4); + assertThat(result.addedManifests()).as("Action should add 2 manifests").hasSize(2); assertManifestsLocation(result.addedManifests()); table.refresh(); List newManifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 2 manifests after rewrite", 2, newManifests.size()); - Assert.assertEquals(4, (long) newManifests.get(0).existingFilesCount()); - Assert.assertFalse(newManifests.get(0).hasAddedFiles()); - Assert.assertFalse(newManifests.get(0).hasDeletedFiles()); + assertThat(newManifests).as("Should have 2 manifests after rewrite").hasSize(2); - Assert.assertEquals(4, (long) newManifests.get(1).existingFilesCount()); - Assert.assertFalse(newManifests.get(1).hasAddedFiles()); - Assert.assertFalse(newManifests.get(1).hasDeletedFiles()); + assertThat(newManifests.get(0).existingFilesCount()).isEqualTo(4); + assertThat(newManifests.get(0).hasAddedFiles()).isFalse(); + assertThat(newManifests.get(0).hasDeletedFiles()).isFalse(); + + assertThat(newManifests.get(1).existingFilesCount()).isEqualTo(4); + assertThat(newManifests.get(1).hasAddedFiles()).isFalse(); + assertThat(newManifests.get(1).hasDeletedFiles()).isFalse(); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records1); @@ -358,10 +356,10 @@ public void testRewriteSmallManifestsPartitionedTable() { List actualRecords = resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); } - @Test + @TestTemplate public void testRewriteImportedManifests() throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); Map options = Maps.newHashMap(); @@ -372,7 +370,7 @@ public void testRewriteImportedManifests() throws IOException { List records = Lists.newArrayList( new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); - File parquetTableDir = temp.newFolder("parquet_table"); + File parquetTableDir = temp.resolve("parquet_table").toFile(); String parquetTableLocation = parquetTableDir.toURI().toString(); try { @@ -386,7 +384,7 @@ public void testRewriteImportedManifests() throws IOException { .partitionBy("c3") .saveAsTable("parquet_table"); - File stagingDir = temp.newFolder("staging-dir"); + File stagingDir = temp.resolve("staging-dir").toFile(); SparkTableUtil.importSparkTable( spark, new TableIdentifier("parquet_table"), table, stagingDir.toString()); @@ -398,7 +396,8 @@ public void testRewriteImportedManifests() throws IOException { SparkActions actions = SparkActions.get(); - String rewriteStagingLocation = temp.newFolder().toString(); + String rewriteStagingLocation = + java.nio.file.Files.createTempDirectory(temp, "junit").toString(); RewriteManifests.Result result = actions @@ -408,12 +407,10 @@ public void testRewriteImportedManifests() throws IOException { .stagingLocation(rewriteStagingLocation) .execute(); - Assert.assertEquals( - "Action should rewrite all manifests", - snapshot.allManifests(table.io()), - result.rewrittenManifests()); - Assert.assertEquals( - "Action should add 1 manifest", 1, Iterables.size(result.addedManifests())); + assertThat(result.rewrittenManifests()) + .as("Action should rewrite all manifests") + .isEqualTo(snapshot.allManifests(table.io())); + assertThat(result.addedManifests()).as("Action should add 1 manifest").hasSize(1); assertManifestsLocation(result.addedManifests(), rewriteStagingLocation); } finally { @@ -421,7 +418,7 @@ public void testRewriteImportedManifests() throws IOException { } } - @Test + @TestTemplate public void testRewriteLargeManifestsPartitionedTable() throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); Map options = Maps.newHashMap(); @@ -437,7 +434,7 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { table.newFastAppend().appendManifest(appendManifest).commit(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 1 manifests before rewrite", 1, manifests.size()); + assertThat(manifests).as("Should have 1 manifests before rewrite").hasSize(1); // set the target manifest size to a small value to force splitting records into multiple files table @@ -449,7 +446,7 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { SparkActions actions = SparkActions.get(); - String stagingLocation = temp.newFolder().toString(); + String stagingLocation = java.nio.file.Files.createTempDirectory(temp, "junit").toString(); RewriteManifests.Result result = actions @@ -459,17 +456,17 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { .stagingLocation(stagingLocation) .execute(); - Assertions.assertThat(result.rewrittenManifests()).hasSize(1); - Assertions.assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); + assertThat(result.rewrittenManifests()).hasSize(1); + assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); assertManifestsLocation(result.addedManifests(), stagingLocation); table.refresh(); List newManifests = table.currentSnapshot().allManifests(table.io()); - Assertions.assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); + assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); } - @Test + @TestTemplate public void testRewriteManifestsWithPredicate() throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 2).build(); Map options = Maps.newHashMap(); @@ -493,11 +490,11 @@ public void testRewriteManifestsWithPredicate() throws IOException { table.refresh(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 3 manifests before rewrite", 3, manifests.size()); + assertThat(manifests).as("Should have 3 manifests before rewrite").hasSize(3); SparkActions actions = SparkActions.get(); - String stagingLocation = temp.newFolder().toString(); + String stagingLocation = java.nio.file.Files.createTempDirectory(temp, "junit").toString(); // rewrite only the first manifest RewriteManifests.Result result = @@ -511,22 +508,22 @@ public void testRewriteManifestsWithPredicate() throws IOException { .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) .execute(); - Assert.assertEquals( - "Action should rewrite 2 manifest", 2, Iterables.size(result.rewrittenManifests())); - Assert.assertEquals( - "Action should add 1 manifests", 1, Iterables.size(result.addedManifests())); + assertThat(result.rewrittenManifests()).as("Action should rewrite 2 manifest").hasSize(2); + assertThat(result.addedManifests()).as("Action should add 1 manifests").hasSize(1); assertManifestsLocation(result.addedManifests(), stagingLocation); table.refresh(); List newManifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 2 manifests after rewrite", 2, newManifests.size()); - - Assert.assertFalse("First manifest must be rewritten", newManifests.contains(manifests.get(0))); - Assert.assertFalse( - "Second manifest must be rewritten", newManifests.contains(manifests.get(1))); - Assert.assertTrue( - "Third manifest must not be rewritten", newManifests.contains(manifests.get(2))); + assertThat(newManifests) + .as("Should have 2 manifests after rewrite") + .hasSize(2) + .as("First manifest must be rewritten") + .doesNotContain(manifests.get(0)) + .as("Second manifest must be rewritten") + .doesNotContain(manifests.get(1)) + .as("Third manifest must not be rewritten") + .contains(manifests.get(2)); List expectedRecords = Lists.newArrayList(); expectedRecords.add(records1.get(0)); @@ -539,10 +536,10 @@ public void testRewriteManifestsWithPredicate() throws IOException { List actualRecords = resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); } - @Test + @TestTemplate public void testRewriteSmallManifestsNonPartitionedV2Table() { assumeThat(formatVersion).isGreaterThan(1); @@ -567,7 +564,7 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { DataFile file2 = Iterables.getOnlyElement(snapshot2.addedDataFiles(table.io())); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 2 manifests before rewrite", 2, manifests.size()); + assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); SparkActions actions = SparkActions.get(); RewriteManifests.Result result = @@ -575,21 +572,19 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { .rewriteManifests(table) .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) .execute(); - Assert.assertEquals( - "Action should rewrite 2 manifests", 2, Iterables.size(result.rewrittenManifests())); - Assert.assertEquals( - "Action should add 1 manifests", 1, Iterables.size(result.addedManifests())); + assertThat(result.rewrittenManifests()).as("Action should rewrite 2 manifests").hasSize(2); + assertThat(result.addedManifests()).as("Action should add 1 manifests").hasSize(1); assertManifestsLocation(result.addedManifests()); table.refresh(); List newManifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 1 manifests after rewrite", 1, newManifests.size()); + assertThat(newManifests).as("Should have 1 manifests after rewrite").hasSize(1); ManifestFile newManifest = Iterables.getOnlyElement(newManifests); - Assert.assertEquals(2, (long) newManifest.existingFilesCount()); - Assert.assertFalse(newManifest.hasAddedFiles()); - Assert.assertFalse(newManifest.hasDeletedFiles()); + assertThat(newManifest.existingFilesCount()).isEqualTo(2); + assertThat(newManifest.hasAddedFiles()).isFalse(); + assertThat(newManifest.hasDeletedFiles()).isFalse(); validateDataManifest( table, @@ -607,10 +602,10 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { List actualRecords = resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); } - @Test + @TestTemplate public void testRewriteLargeManifestsEvolvedUnpartitionedV1Table() throws IOException { assumeThat(formatVersion).isEqualTo(1); @@ -659,7 +654,7 @@ public void testRewriteLargeManifestsEvolvedUnpartitionedV1Table() throws IOExce assertThat(manifests).hasSizeGreaterThanOrEqualTo(2); } - @Test + @TestTemplate public void testRewriteSmallDeleteManifestsNonPartitionedTable() throws IOException { assumeThat(formatVersion).isGreaterThan(1); @@ -732,7 +727,7 @@ public void testRewriteSmallDeleteManifestsNonPartitionedTable() throws IOExcept assertThat(actualRecords()).isEqualTo(expectedRecords); } - @Test + @TestTemplate public void testRewriteSmallDeleteManifestsPartitionedTable() throws IOException { assumeThat(formatVersion).isGreaterThan(1); @@ -835,7 +830,7 @@ public void testRewriteSmallDeleteManifestsPartitionedTable() throws IOException assertThat(actualRecords()).isEqualTo(expectedRecords); } - @Test + @TestTemplate public void testRewriteLargeDeleteManifestsPartitionedTable() throws IOException { assumeThat(formatVersion).isGreaterThan(1); @@ -874,7 +869,7 @@ public void testRewriteLargeDeleteManifestsPartitionedTable() throws IOException SparkActions actions = SparkActions.get(); - String stagingLocation = temp.newFolder().toString(); + String stagingLocation = java.nio.file.Files.createTempDirectory(temp, "junit").toString(); RewriteManifests.Result result = actions @@ -948,8 +943,8 @@ private void assertManifestsLocation(Iterable manifests, String st } private ManifestFile writeManifest(Table table, List files) throws IOException { - File manifestFile = temp.newFile("generated-manifest.avro"); - Assert.assertTrue(manifestFile.delete()); + File manifestFile = File.createTempFile("generated-manifest", ".avro", temp.toFile()); + assertThat(manifestFile.delete()).isTrue(); OutputFile outputFile = table.io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = @@ -1018,7 +1013,7 @@ private Pair writePosDeletes( private Pair writePosDeletes( Table table, StructLike partition, List> deletes) throws IOException { - OutputFile outputFile = Files.localOutput(temp.newFile()); + OutputFile outputFile = Files.localOutput(File.createTempFile("junit", null, temp.toFile())); return FileHelpers.writeDeleteFile(table, outputFile, partition, deletes); } @@ -1036,7 +1031,7 @@ private DeleteFile writeEqDeletes(Table table, StructLike partition, String key, deletes.add(delete.copy(key, value)); } - OutputFile outputFile = Files.localOutput(temp.newFile()); + OutputFile outputFile = Files.localOutput(File.createTempFile("junit", null, temp.toFile())); return FileHelpers.writeDeleteFile(table, outputFile, partition, deletes, deleteSchema); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 9149bb7652dc..d1e33950ebe9 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -19,8 +19,12 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; +import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -36,6 +40,8 @@ import org.apache.iceberg.Files; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Partitioning; @@ -59,8 +65,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalogConfig; -import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.FourColumnRecord; import org.apache.iceberg.spark.source.ThreeColumnRecord; @@ -69,14 +75,11 @@ import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.junit.After; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.io.TempDir; -public class TestRewritePositionDeleteFilesAction extends SparkCatalogTestBase { +public class TestRewritePositionDeleteFilesAction extends CatalogTestBase { private static final String TABLE_NAME = "test_table"; private static final Schema SCHEMA = @@ -94,9 +97,7 @@ public class TestRewritePositionDeleteFilesAction extends SparkCatalogTestBase { private static final int SCALE = 4000; private static final int DELETES_SCALE = 1000; - @Parameterized.Parameters( - name = - "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}, fileFormat = {4}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, fileFormat = {3}") public static Object[][] parameters() { return new Object[][] { { @@ -108,22 +109,17 @@ public static Object[][] parameters() { }; } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - private final FileFormat format; + @Parameter(index = 3) + private FileFormat format; - public TestRewritePositionDeleteFilesAction( - String catalogName, String implementation, Map config, FileFormat format) { - super(catalogName, implementation, config); - this.format = format; - } - - @After + @AfterEach public void cleanup() { validationCatalog.dropTable(TableIdentifier.of("default", TABLE_NAME)); } - @Test + @TestTemplate public void testEmptyTable() { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); Table table = @@ -131,24 +127,24 @@ public void testEmptyTable() { TableIdentifier.of("default", TABLE_NAME), SCHEMA, spec, tableProperties()); Result result = SparkActions.get(spark).rewritePositionDeletes(table).execute(); - Assert.assertEquals("No rewritten delete files", 0, result.rewrittenDeleteFilesCount()); - Assert.assertEquals("No added delete files", 0, result.addedDeleteFilesCount()); + assertThat(result.rewrittenDeleteFilesCount()).as("No rewritten delete files").isZero(); + assertThat(result.addedDeleteFilesCount()).as("No added delete files").isZero(); } - @Test + @TestTemplate public void testUnpartitioned() throws Exception { Table table = createTableUnpartitioned(2, SCALE); List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); - Assert.assertEquals(2, dataFiles.size()); + assertThat(dataFiles).hasSize(2); List deleteFiles = deleteFiles(table); - Assert.assertEquals(2, deleteFiles.size()); + assertThat(deleteFiles).hasSize(2); List expectedRecords = records(table); List expectedDeletes = deleteRecords(table); - Assert.assertEquals(2000, expectedRecords.size()); - Assert.assertEquals(2000, expectedDeletes.size()); + assertThat(expectedRecords).hasSize(2000); + assertThat(expectedDeletes).hasSize(2000); Result result = SparkActions.get(spark) @@ -156,7 +152,7 @@ public void testUnpartitioned() throws Exception { .option(SizeBasedFileRewriter.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); - Assert.assertEquals("Expected 1 new delete file", 1, newDeleteFiles.size()); + assertThat(newDeleteFiles).as("Expected 1 new delete file").hasSize(1); assertLocallySorted(newDeleteFiles); assertNotContains(deleteFiles, newDeleteFiles); checkResult(result, deleteFiles, newDeleteFiles, 1); @@ -168,21 +164,21 @@ public void testUnpartitioned() throws Exception { assertEquals("Position deletes must match", expectedDeletes, actualDeletes); } - @Test + @TestTemplate public void testRewriteAll() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); - Assert.assertEquals(4, dataFiles.size()); + assertThat(dataFiles).hasSize(4); List deleteFiles = deleteFiles(table); - Assert.assertEquals(8, deleteFiles.size()); + assertThat(deleteFiles).hasSize(8); List expectedRecords = records(table); List expectedDeletes = deleteRecords(table); - Assert.assertEquals(12000, expectedRecords.size()); - Assert.assertEquals(4000, expectedDeletes.size()); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); Result result = SparkActions.get(spark) @@ -192,7 +188,7 @@ public void testRewriteAll() throws Exception { .execute(); List newDeleteFiles = deleteFiles(table); - Assert.assertEquals("Should have 4 delete files", 4, newDeleteFiles.size()); + assertThat(newDeleteFiles).hasSize(4); assertNotContains(deleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, deleteFiles, newDeleteFiles, 4); @@ -204,23 +200,23 @@ public void testRewriteAll() throws Exception { assertEquals("Position deletes must match", expectedDeletes, actualDeletes); } - @Test + @TestTemplate public void testRewriteFilter() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); table.refresh(); List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); - Assert.assertEquals(4, dataFiles.size()); + assertThat(dataFiles).hasSize(4); List deleteFiles = deleteFiles(table); - Assert.assertEquals(8, deleteFiles.size()); + assertThat(deleteFiles).hasSize(8); table.refresh(); List expectedRecords = records(table); List expectedDeletes = deleteRecords(table); - Assert.assertEquals(12000, expectedRecords.size()); // 16000 data - 4000 delete rows - Assert.assertEquals(4000, expectedDeletes.size()); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); Expression filter = Expressions.and( @@ -236,7 +232,7 @@ public void testRewriteFilter() throws Exception { .execute(); List newDeleteFiles = except(deleteFiles(table), deleteFiles); - Assert.assertEquals("Should have 4 delete files", 2, newDeleteFiles.size()); + assertThat(newDeleteFiles).as("Should have 4 delete files").hasSize(2); List expectedRewrittenFiles = filterFiles(table, deleteFiles, ImmutableList.of(1), ImmutableList.of(2)); @@ -249,21 +245,21 @@ public void testRewriteFilter() throws Exception { assertEquals("Position deletes must match", expectedDeletes, actualDeletes); } - @Test + @TestTemplate public void testRewriteToSmallerTarget() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); - Assert.assertEquals(4, dataFiles.size()); + assertThat(dataFiles).hasSize(4); List expectedRecords = records(table); List expectedDeletes = deleteRecords(table); - Assert.assertEquals(12000, expectedRecords.size()); - Assert.assertEquals(4000, expectedDeletes.size()); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); List deleteFiles = deleteFiles(table); - Assert.assertEquals(8, deleteFiles.size()); + assertThat(deleteFiles).hasSize(8); long avgSize = size(deleteFiles) / deleteFiles.size(); @@ -274,7 +270,7 @@ public void testRewriteToSmallerTarget() throws Exception { .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, String.valueOf(avgSize / 2)) .execute(); List newDeleteFiles = deleteFiles(table); - Assert.assertEquals("Should have 8 new delete files", 8, newDeleteFiles.size()); + assertThat(newDeleteFiles).as("Should have 8 new delete files").hasSize(8); assertNotContains(deleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, deleteFiles, newDeleteFiles, 4); @@ -286,7 +282,7 @@ public void testRewriteToSmallerTarget() throws Exception { assertEquals("Position deletes must match", expectedDeletes, actualDeletes); } - @Test + @TestTemplate public void testRemoveDanglingDeletes() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); @@ -298,15 +294,15 @@ public void testRemoveDanglingDeletes() throws Exception { dataFiles, true /* Disable commit-time ManifestFilterManager removal of dangling deletes */); - Assert.assertEquals(4, dataFiles.size()); + assertThat(dataFiles).hasSize(4); List deleteFiles = deleteFiles(table); - Assert.assertEquals(8, deleteFiles.size()); + assertThat(deleteFiles).hasSize(8); List expectedRecords = records(table); List expectedDeletes = deleteRecords(table); - Assert.assertEquals(12000, expectedRecords.size()); - Assert.assertEquals(4000, expectedDeletes.size()); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); SparkActions.get(spark) .rewriteDataFiles(table) @@ -319,7 +315,7 @@ public void testRemoveDanglingDeletes() throws Exception { .option(SizeBasedFileRewriter.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); - Assert.assertEquals("Should have 0 new delete files", 0, newDeleteFiles.size()); + assertThat(newDeleteFiles).as("Should have 0 new delete files").hasSize(0); assertNotContains(deleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, deleteFiles, newDeleteFiles, 4); @@ -328,24 +324,24 @@ public void testRemoveDanglingDeletes() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); assertEquals("Rows must match", expectedRecords, actualRecords); - Assert.assertEquals("Should be no new position deletes", 0, actualDeletes.size()); + assertThat(actualDeletes).as("Should be no new position deletes").hasSize(0); } - @Test + @TestTemplate public void testSomePartitionsDanglingDeletes() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); - Assert.assertEquals(4, dataFiles.size()); + assertThat(dataFiles).hasSize(4); List deleteFiles = deleteFiles(table); - Assert.assertEquals(8, deleteFiles.size()); + assertThat(deleteFiles).hasSize(8); List expectedRecords = records(table); List expectedDeletes = deleteRecords(table); - Assert.assertEquals(12000, expectedRecords.size()); - Assert.assertEquals(4000, expectedDeletes.size()); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); // Rewrite half the data files Expression filter = Expressions.or(Expressions.equal("c1", 0), Expressions.equal("c1", 1)); @@ -361,7 +357,7 @@ public void testSomePartitionsDanglingDeletes() throws Exception { .option(SizeBasedFileRewriter.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); - Assert.assertEquals("Should have 2 new delete files", 2, newDeleteFiles.size()); + assertThat(newDeleteFiles).as("Should have 2 new delete files").hasSize(2); assertNotContains(deleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, deleteFiles, newDeleteFiles, 4); @@ -384,23 +380,23 @@ public void testSomePartitionsDanglingDeletes() throws Exception { assertEquals("Position deletes must match", expectedDeletes, actualDeletes); } - @Test + @TestTemplate public void testRewriteFilterRemoveDangling() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); table.refresh(); List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles, true); - Assert.assertEquals(4, dataFiles.size()); + assertThat(dataFiles).hasSize(4); List deleteFiles = deleteFiles(table); - Assert.assertEquals(8, deleteFiles.size()); + assertThat(deleteFiles).hasSize(8); table.refresh(); List expectedRecords = records(table); List expectedDeletes = deleteRecords(table); - Assert.assertEquals(12000, expectedRecords.size()); // 16000 data - 4000 delete rows - Assert.assertEquals(4000, expectedDeletes.size()); + assertThat(expectedRecords).hasSize(12000); // 16000 data - 4000 delete rows + assertThat(expectedDeletes).hasSize(4000); SparkActions.get(spark) .rewriteDataFiles(table) @@ -417,7 +413,7 @@ public void testRewriteFilterRemoveDangling() throws Exception { .execute(); List newDeleteFiles = except(deleteFiles(table), deleteFiles); - Assert.assertEquals("Should have 2 new delete files", 0, newDeleteFiles.size()); + assertThat(newDeleteFiles).as("Should have 2 new delete files").hasSize(0); List expectedRewrittenFiles = filterFiles(table, deleteFiles, ImmutableList.of(0), ImmutableList.of(1)); @@ -432,35 +428,35 @@ public void testRewriteFilterRemoveDangling() throws Exception { assertEquals("Position deletes must match", expectedDeletesFiltered, allDeletes); } - @Test + @TestTemplate public void testPartitionEvolutionAdd() throws Exception { Table table = createTableUnpartitioned(2, SCALE); List unpartitionedDataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, unpartitionedDataFiles); - Assert.assertEquals(2, unpartitionedDataFiles.size()); + assertThat(unpartitionedDataFiles).hasSize(2); List unpartitionedDeleteFiles = deleteFiles(table); - Assert.assertEquals(2, unpartitionedDeleteFiles.size()); + assertThat(unpartitionedDeleteFiles).hasSize(2); List expectedUnpartitionedDeletes = deleteRecords(table); List expectedUnpartitionedRecords = records(table); - Assert.assertEquals(2000, expectedUnpartitionedRecords.size()); - Assert.assertEquals(2000, expectedUnpartitionedDeletes.size()); + assertThat(expectedUnpartitionedRecords).hasSize(2000); + assertThat(expectedUnpartitionedDeletes).hasSize(2000); table.updateSpec().addField("c1").commit(); writeRecords(table, 2, SCALE, 2); List partitionedDataFiles = except(TestHelpers.dataFiles(table), unpartitionedDataFiles); writePosDeletesForFiles(table, 2, DELETES_SCALE, partitionedDataFiles); - Assert.assertEquals(2, partitionedDataFiles.size()); + assertThat(partitionedDataFiles).hasSize(2); List partitionedDeleteFiles = except(deleteFiles(table), unpartitionedDeleteFiles); - Assert.assertEquals(4, partitionedDeleteFiles.size()); + assertThat(partitionedDeleteFiles).hasSize(4); List expectedDeletes = deleteRecords(table); List expectedRecords = records(table); - Assert.assertEquals(4000, expectedDeletes.size()); - Assert.assertEquals(8000, expectedRecords.size()); + assertThat(expectedDeletes).hasSize(4000); + assertThat(expectedRecords).hasSize(8000); Result result = SparkActions.get(spark) @@ -472,7 +468,7 @@ public void testPartitionEvolutionAdd() throws Exception { Stream.concat(unpartitionedDeleteFiles.stream(), partitionedDeleteFiles.stream()) .collect(Collectors.toList()); List newDeleteFiles = deleteFiles(table); - Assert.assertEquals("Should have 3 new delete files", 3, newDeleteFiles.size()); + assertThat(newDeleteFiles).as("Should have 3 new delete files").hasSize(3); assertNotContains(rewrittenDeleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, rewrittenDeleteFiles, newDeleteFiles, 3); @@ -484,15 +480,15 @@ public void testPartitionEvolutionAdd() throws Exception { assertEquals("Position deletes must match", expectedDeletes, actualDeletes); } - @Test + @TestTemplate public void testPartitionEvolutionRemove() throws Exception { Table table = createTablePartitioned(2, 2, SCALE); List dataFilesUnpartitioned = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFilesUnpartitioned); - Assert.assertEquals(2, dataFilesUnpartitioned.size()); + assertThat(dataFilesUnpartitioned).hasSize(2); List deleteFilesUnpartitioned = deleteFiles(table); - Assert.assertEquals(4, deleteFilesUnpartitioned.size()); + assertThat(deleteFilesUnpartitioned).hasSize(4); table.updateSpec().removeField("c1").commit(); @@ -500,18 +496,18 @@ public void testPartitionEvolutionRemove() throws Exception { List dataFilesPartitioned = except(TestHelpers.dataFiles(table), dataFilesUnpartitioned); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFilesPartitioned); - Assert.assertEquals(2, dataFilesPartitioned.size()); + assertThat(dataFilesPartitioned).hasSize(2); List deleteFilesPartitioned = except(deleteFiles(table), deleteFilesUnpartitioned); - Assert.assertEquals(2, deleteFilesPartitioned.size()); + assertThat(deleteFilesPartitioned).hasSize(2); List expectedRecords = records(table); List expectedDeletes = deleteRecords(table); - Assert.assertEquals(4000, expectedDeletes.size()); - Assert.assertEquals(8000, expectedRecords.size()); + assertThat(expectedDeletes).hasSize(4000); + assertThat(expectedRecords).hasSize(8000); List expectedRewritten = deleteFiles(table); - Assert.assertEquals(6, expectedRewritten.size()); + assertThat(expectedRewritten).hasSize(6); Result result = SparkActions.get(spark) @@ -519,7 +515,7 @@ public void testPartitionEvolutionRemove() throws Exception { .option(SizeBasedFileRewriter.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); - Assert.assertEquals("Should have 3 new delete files", 3, newDeleteFiles.size()); + assertThat(newDeleteFiles).as("Should have 3 new delete files").hasSize(3); assertNotContains(expectedRewritten, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, expectedRewritten, newDeleteFiles, 3); @@ -531,15 +527,15 @@ public void testPartitionEvolutionRemove() throws Exception { assertEquals("Position deletes must match", expectedDeletes, actualDeletes); } - @Test + @TestTemplate public void testSchemaEvolution() throws Exception { Table table = createTablePartitioned(2, 2, SCALE); List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); - Assert.assertEquals(2, dataFiles.size()); + assertThat(dataFiles).hasSize(2); List deleteFiles = deleteFiles(table); - Assert.assertEquals(4, deleteFiles.size()); + assertThat(deleteFiles).hasSize(4); table.updateSchema().addColumn("c4", Types.StringType.get()).commit(); writeNewSchemaRecords(table, 2, SCALE, 2, 2); @@ -552,13 +548,13 @@ public void testSchemaEvolution() throws Exception { writePosDeletesForFiles(table, 2, DELETES_SCALE, newSchemaDataFiles); List newSchemaDeleteFiles = except(deleteFiles(table), deleteFiles); - Assert.assertEquals(4, newSchemaDeleteFiles.size()); + assertThat(newSchemaDeleteFiles).hasSize(4); table.refresh(); List expectedDeletes = deleteRecords(table); List expectedRecords = records(table); - Assert.assertEquals(4000, expectedDeletes.size()); // 4 files * 1000 per file - Assert.assertEquals(12000, expectedRecords.size()); // 4 * 4000 - 4000 + assertThat(expectedDeletes).hasSize(4000); // 4 files * 1000 per file + assertThat(expectedRecords).hasSize(12000); // 4 * 4000 - 4000 Result result = SparkActions.get(spark) @@ -570,7 +566,7 @@ public void testSchemaEvolution() throws Exception { Stream.concat(deleteFiles.stream(), newSchemaDeleteFiles.stream()) .collect(Collectors.toList()); List newDeleteFiles = deleteFiles(table); - Assert.assertEquals("Should have 2 new delete files", 4, newDeleteFiles.size()); + assertThat(newDeleteFiles).as("Should have 2 new delete files").hasSize(4); assertNotContains(rewrittenDeleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, rewrittenDeleteFiles, newDeleteFiles, 4); @@ -627,9 +623,10 @@ private void writeRecords(Table table, int files, int numRecords, int numPartiti private void writeRecordsWithPartitions( Table table, int files, int numRecords, List> partitions) { int partitionTypeSize = table.spec().partitionType().fields().size(); - Assert.assertTrue( - "This method currently supports only two columns as partition columns", - partitionTypeSize <= 2); + assertThat(partitionTypeSize) + .as("This method currently supports only two columns as partition columns") + .isLessThanOrEqualTo(2); + BiFunction, ThreeColumnRecord> recordFunction = (i, partValues) -> { switch (partitionTypeSize) { @@ -737,13 +734,13 @@ private void writePosDeletesForFiles( List partitionFiles = filesByPartitionEntry.getValue(); int deletesForPartition = partitionFiles.size() * deletesPerDataFile; - Assert.assertEquals( - "Number of delete files per partition should be " - + "evenly divisible by requested deletes per data file times number of data files in this partition", - 0, - deletesForPartition % deleteFilesPerPartition); - int deleteFileSize = deletesForPartition / deleteFilesPerPartition; + assertThat(deletesForPartition % deleteFilesPerPartition) + .as( + "Number of delete files per partition should be " + + "evenly divisible by requested deletes per data file times number of data files in this partition") + .isZero(); + int deleteFileSize = deletesForPartition / deleteFilesPerPartition; int counter = 0; List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { @@ -752,7 +749,8 @@ private void writePosDeletesForFiles( counter++; if (counter == deleteFileSize) { // Dump to file and reset variables - OutputFile output = Files.localOutput(temp.newFile()); + OutputFile output = + Files.localOutput(File.createTempFile("junit", null, temp.toFile())); deleteFiles.add(FileHelpers.writeDeleteFile(table, output, partition, deletes).first()); counter = 0; deletes.clear(); @@ -797,7 +795,7 @@ private void assertNotContains(List original, List rewri Set rewrittenPaths = rewritten.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); rewrittenPaths.retainAll(originalPaths); - Assert.assertEquals(0, rewrittenPaths.size()); + assertThat(rewrittenPaths).hasSize(0); } private void assertLocallySorted(List deleteFiles) { @@ -806,16 +804,16 @@ private void assertLocallySorted(List deleteFiles) { spark.read().format("iceberg").load("default." + TABLE_NAME + ".position_deletes"); deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.path().toString())); List rows = deletes.collectAsList(); - Assert.assertFalse("Empty delete file found", rows.isEmpty()); + assertThat(rows).as("Empty delete file found").isNotEmpty(); int lastPos = 0; String lastPath = ""; for (Row row : rows) { String path = row.getAs("file_path"); long pos = row.getAs("pos"); if (path.compareTo(lastPath) < 0) { - Assert.fail(String.format("File_path not sorted, Found %s after %s", path, lastPath)); + fail(String.format("File_path not sorted, Found %s after %s", path, lastPath)); } else if (path.equals(lastPath)) { - Assert.assertTrue("Pos not sorted", pos >= lastPos); + assertThat(pos).as("Pos not sorted").isGreaterThanOrEqualTo(lastPos); } } } @@ -823,7 +821,8 @@ private void assertLocallySorted(List deleteFiles) { private String name(Table table) { String[] splits = table.name().split("\\."); - Assert.assertEquals(3, splits.length); + + assertThat(splits).hasSize(3); return String.format("%s.%s", splits[1], splits[2]); } @@ -901,49 +900,53 @@ private void checkResult( List rewrittenDeletes, List newDeletes, int expectedGroups) { - Assert.assertEquals( - "Expected rewritten delete file count does not match", - rewrittenDeletes.size(), - result.rewrittenDeleteFilesCount()); - Assert.assertEquals( - "Expected new delete file count does not match", - newDeletes.size(), - result.addedDeleteFilesCount()); - Assert.assertEquals( - "Expected rewritten delete byte count does not match", - size(rewrittenDeletes), - result.rewrittenBytesCount()); - Assert.assertEquals( - "Expected new delete byte count does not match", - size(newDeletes), - result.addedBytesCount()); - - Assert.assertEquals( - "Expected rewrite group count does not match", - expectedGroups, - result.rewriteResults().size()); - Assert.assertEquals( - "Expected rewritten delete file count in all groups to match", - rewrittenDeletes.size(), - result.rewriteResults().stream() - .mapToInt(FileGroupRewriteResult::rewrittenDeleteFilesCount) - .sum()); - Assert.assertEquals( - "Expected added delete file count in all groups to match", - newDeletes.size(), - result.rewriteResults().stream() - .mapToInt(FileGroupRewriteResult::addedDeleteFilesCount) - .sum()); - Assert.assertEquals( - "Expected rewritten delete bytes in all groups to match", - size(rewrittenDeletes), - result.rewriteResults().stream() - .mapToLong(FileGroupRewriteResult::rewrittenBytesCount) - .sum()); - Assert.assertEquals( - "Expected added delete bytes in all groups to match", - size(newDeletes), - result.rewriteResults().stream().mapToLong(FileGroupRewriteResult::addedBytesCount).sum()); + assertThat(rewrittenDeletes.size()) + .as("Expected rewritten delete file count does not match") + .isEqualTo(result.rewrittenDeleteFilesCount()); + + assertThat(newDeletes.size()) + .as("Expected new delete file count does not match") + .isEqualTo(result.addedDeleteFilesCount()); + + assertThat(size(rewrittenDeletes)) + .as("Expected rewritten delete byte count does not match") + .isEqualTo(result.rewrittenBytesCount()); + + assertThat(size(newDeletes)) + .as("Expected new delete byte count does not match") + .isEqualTo(result.addedBytesCount()); + + assertThat(expectedGroups) + .as("Expected rewrite group count does not match") + .isEqualTo(result.rewriteResults().size()); + + assertThat(rewrittenDeletes.size()) + .as("Expected rewritten delete file count in all groups to match") + .isEqualTo( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::rewrittenDeleteFilesCount) + .sum()); + + assertThat(newDeletes.size()) + .as("Expected added delete file count in all groups to match") + .isEqualTo( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::addedDeleteFilesCount) + .sum()); + + assertThat(size(rewrittenDeletes)) + .as("Expected rewritten delete bytes in all groups to match") + .isEqualTo( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::rewrittenBytesCount) + .sum()); + + assertThat(size(newDeletes)) + .as("Expected added delete bytes in all groups to match") + .isEqualTo( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::addedBytesCount) + .sum()); } private void checkSequenceNumbers( @@ -961,10 +964,9 @@ private void checkSequenceNumbers( if (addedPartitionFiles != null) { addedPartitionFiles.forEach( d -> - Assert.assertEquals( - "Sequence number should be max of rewritten set", - d.dataSequenceNumber(), - maxRewrittenSeq)); + assertThat(d.dataSequenceNumber()) + .as("Sequence number should be max of rewritten set") + .isEqualTo(maxRewrittenSeq)); } } }