From 2d41bef20559e4ac2a175cd8469cd295a21df9c6 Mon Sep 17 00:00:00 2001 From: Tomohiro Tanaka Date: Tue, 20 Feb 2024 20:42:29 +0900 Subject: [PATCH 1/6] Migrate Procedure sub-classes in spark-extensions to JUnit5 and AssertJ style --- .../extensions/TestAncestorsOfProcedure.java | 27 ++--- .../extensions/TestCallStatementParser.java | 65 +++++------ .../extensions/TestMigrateTableProcedure.java | 110 +++++++++--------- .../TestRegisterTableProcedure.java | 43 ++++--- .../TestSnapshotTableProcedure.java | 91 ++++++++------- 5 files changed, 163 insertions(+), 173 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java index 3282cdf381ce..437b4c368232 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java @@ -19,27 +19,24 @@ package org.apache.iceberg.spark.extensions; import java.util.List; -import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestAncestorsOfProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestAncestorsOfProcedure extends ExtensionsTestBase { - public TestAncestorsOfProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testAncestorOfUsingEmptyArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -60,7 +57,7 @@ public void testAncestorOfUsingEmptyArgs() { output); } - @Test + @TestTemplate public void testAncestorOfUsingSnapshotId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -84,7 +81,7 @@ public void testAncestorOfUsingSnapshotId() { sql("CALL %s.system.ancestors_of('%s', %dL)", catalogName, tableIdent, preSnapshotId)); } - @Test + @TestTemplate public void testAncestorOfWithRollBack() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); Table table = validationCatalog.loadTable(tableIdent); @@ -128,7 +125,7 @@ public void testAncestorOfWithRollBack() { sql("CALL %s.system.ancestors_of('%s', %dL)", catalogName, tableIdent, thirdSnapshotId)); } - @Test + @TestTemplate public void testAncestorOfUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -145,7 +142,7 @@ public void testAncestorOfUsingNamedArgs() { catalogName, firstSnapshotId, tableIdent)); } - @Test + @TestTemplate public void testInvalidAncestorOfCases() { Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java index 5c41604afae1..14e78173e7f9 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; +import static scala.collection.JavaConverters.seqAsJavaList; + import java.math.BigDecimal; import java.sql.Timestamp; import java.time.Instant; @@ -38,22 +41,16 @@ import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import scala.collection.JavaConverters; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; public class TestCallStatementParser { - @Rule public TemporaryFolder temp = new TemporaryFolder(); - private static SparkSession spark = null; private static ParserInterface parser = null; - @BeforeClass + @BeforeAll public static void startSpark() { TestCallStatementParser.spark = SparkSession.builder() @@ -64,7 +61,7 @@ public static void startSpark() { TestCallStatementParser.parser = spark.sessionState().sqlParser(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestCallStatementParser.spark; TestCallStatementParser.spark = null; @@ -76,10 +73,9 @@ public static void stopSpark() { public void testCallWithPositionalArgs() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL c.n.func(1, '2', 3L, true, 1.0D, 9.0e1, 900e-1BD)"); - Assert.assertEquals( - ImmutableList.of("c", "n", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())).hasSameElementsAs(ImmutableList.of("c", "n", "func")); - Assert.assertEquals(7, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(7); checkArg(call, 0, 1, DataTypes.IntegerType); checkArg(call, 1, "2", DataTypes.StringType); @@ -94,10 +90,10 @@ public void testCallWithPositionalArgs() throws ParseException { public void testCallWithNamedArgs() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, c2 => '2', c3 => true)"); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())) + .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); - Assert.assertEquals(3, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(3); checkArg(call, 0, "c1", 1, DataTypes.IntegerType); checkArg(call, 1, "c2", "2", DataTypes.StringType); @@ -107,10 +103,10 @@ public void testCallWithNamedArgs() throws ParseException { @Test public void testCallWithMixedArgs() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, '2')"); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())) + .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); - Assert.assertEquals(2, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(2); checkArg(call, 0, "c1", 1, DataTypes.IntegerType); checkArg(call, 1, "2", DataTypes.StringType); @@ -121,10 +117,10 @@ public void testCallWithTimestampArg() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(TIMESTAMP '2017-02-03T10:37:30.00Z')"); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())) + .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); - Assert.assertEquals(1, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(1); checkArg( call, 0, Timestamp.from(Instant.parse("2017-02-03T10:37:30.00Z")), DataTypes.TimestampType); @@ -134,10 +130,10 @@ public void testCallWithTimestampArg() throws ParseException { public void testCallWithVarSubstitution() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func('${spark.extra.prop}')"); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())) + .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); - Assert.assertEquals(1, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(1); checkArg(call, 0, "value", DataTypes.StringType); } @@ -165,10 +161,10 @@ public void testCallStripsComments() throws ParseException { "CALL -- a line ending comment\n" + "cat.system.func('${spark.extra.prop}')"); for (String sqlText : callStatementsWithComments) { CallStatement call = (CallStatement) parser.parsePlan(sqlText); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())) + .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); - Assert.assertEquals(1, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(1); checkArg(call, 0, "value", DataTypes.StringType); } @@ -188,7 +184,7 @@ private void checkArg( if (expectedName != null) { NamedArgument arg = checkCast(call.args().apply(index), NamedArgument.class); - Assert.assertEquals(expectedName, arg.name()); + assertThat(arg.name()).isEqualTo(expectedName); } else { CallArgument arg = call.args().apply(index); checkCast(arg, PositionalArgument.class); @@ -196,8 +192,8 @@ private void checkArg( Expression expectedExpr = toSparkLiteral(expectedValue, expectedType); Expression actualExpr = call.args().apply(index).expr(); - Assert.assertEquals("Arg types must match", expectedExpr.dataType(), actualExpr.dataType()); - Assert.assertEquals("Arg must match", expectedExpr, actualExpr); + assertThat(actualExpr.dataType()).as("Arg types must match").isEqualTo(expectedExpr.dataType()); + assertThat(actualExpr).as("Arg must match").isEqualTo(expectedExpr); } private Literal toSparkLiteral(Object value, DataType dataType) { @@ -205,8 +201,9 @@ private Literal toSparkLiteral(Object value, DataType dataType) { } private T checkCast(Object value, Class expectedClass) { - Assert.assertTrue( - "Expected instance of " + expectedClass.getName(), expectedClass.isInstance(value)); + assertThat(expectedClass.isInstance(value)) + .as("Expected instance of " + expectedClass.getName()) + .isTrue(); return expectedClass.cast(value); } } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index 0d23e4bbf068..7d98cc83c841 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -18,51 +18,45 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.io.IOException; import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestMigrateTableProcedure extends SparkExtensionsTestBase { - - public TestMigrateTableProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Rule public TemporaryFolder temp = new TemporaryFolder(); +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; - @After +@ExtendWith(ParameterizedTestExtension.class) +public class TestMigrateTableProcedure extends ExtensionsTestBase { + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s_BACKUP_", tableName); } - @Test + @TestTemplate public void testMigrate() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); Table createdTable = validationCatalog.loadTable(tableIdent); String tableLocation = createdTable.location().replace("file:", ""); - Assert.assertEquals("Table should have original location", location, tableLocation); + assertThat(tableLocation).as("Table should have original location").isEqualTo(location); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -74,10 +68,10 @@ public void testMigrate() throws IOException { sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); } - @Test + @TestTemplate public void testMigrateWithOptions() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -86,15 +80,15 @@ public void testMigrateWithOptions() throws IOException { Object result = scalarSql("CALL %s.system.migrate('%s', map('foo', 'bar'))", catalogName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); Table createdTable = validationCatalog.loadTable(tableIdent); Map props = createdTable.properties(); - Assert.assertEquals("Should have extra property set", "bar", props.get("foo")); + assertThat(props.get("foo")).as("Should have extra property set").isEqualTo("bar"); String tableLocation = createdTable.location().replace("file:", ""); - Assert.assertEquals("Table should have original location", location, tableLocation); + assertThat(tableLocation).as("Table should have original location").isEqualTo(location); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -106,10 +100,10 @@ public void testMigrateWithOptions() throws IOException { sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); } - @Test + @TestTemplate public void testMigrateWithDropBackup() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -118,14 +112,14 @@ public void testMigrateWithDropBackup() throws IOException { Object result = scalarSql( "CALL %s.system.migrate(table => '%s', drop_backup => true)", catalogName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); - Assert.assertFalse(spark.catalog().tableExists(tableName + "_BACKUP_")); + assertThat(result).as("Should have added one file").isEqualTo(1L); + assertThat(spark.catalog().tableExists(tableName + "_BACKUP_")).isFalse(); } - @Test + @TestTemplate public void testMigrateWithBackupTableName() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -137,16 +131,16 @@ public void testMigrateWithBackupTableName() throws IOException { "CALL %s.system.migrate(table => '%s', backup_table_name => '%s')", catalogName, tableName, backupTableName); - Assertions.assertThat(result).isEqualTo(1L); + assertThat(result).isEqualTo(1L); String dbName = tableName.split("\\.")[0]; - Assertions.assertThat(spark.catalog().tableExists(dbName + "." + backupTableName)).isTrue(); + assertThat(spark.catalog().tableExists(dbName + "." + backupTableName)).isTrue(); } - @Test + @TestTemplate public void testMigrateWithInvalidMetricsConfig() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.newFolder().toString(); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -160,11 +154,11 @@ public void testMigrateWithInvalidMetricsConfig() throws IOException { .hasMessageStartingWith("Invalid metrics config"); } - @Test + @TestTemplate public void testMigrateWithConflictingProps() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.newFolder().toString(); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -172,7 +166,7 @@ public void testMigrateWithConflictingProps() throws IOException { Object result = scalarSql("CALL %s.system.migrate('%s', map('migrated', 'false'))", catalogName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); assertEquals( "Should have expected rows", @@ -180,10 +174,12 @@ public void testMigrateWithConflictingProps() throws IOException { sql("SELECT * FROM %s", tableName)); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should override user value", "true", table.properties().get("migrated")); + assertThat(table.properties().get("migrated")) + .as("Should override user value") + .isEqualTo("true"); } - @Test + @TestTemplate public void testInvalidMigrateCases() { Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) .isInstanceOf(AnalysisException.class) @@ -199,10 +195,10 @@ public void testInvalidMigrateCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testMigratePartitionWithSpecialCharacter() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string, dt date) USING parquet " + "PARTITIONED BY (data, dt) LOCATION '%s'", @@ -216,25 +212,25 @@ public void testMigratePartitionWithSpecialCharacter() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testMigrateEmptyPartitionedTable() throws Exception { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", tableName, location); Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); - Assert.assertEquals(0L, result); + assertThat(result).isEqualTo(0L); } - @Test + @TestTemplate public void testMigrateEmptyTable() throws Exception { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); - Assert.assertEquals(0L, result); + assertThat(result).isEqualTo(0L); } } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java index 2f1165e9cd5e..3047dccd959b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java @@ -18,40 +18,39 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; -import java.util.Map; import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.spark.Spark3Util; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.functions; import org.apache.spark.sql.types.DataTypes; -import org.junit.After; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestRegisterTableProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestRegisterTableProcedure extends ExtensionsTestBase { - private final String targetName; + private String targetName; - public TestRegisterTableProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); + @BeforeEach + public void setTargetName() { targetName = tableName("register_table"); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - @After + @AfterEach public void dropTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s", targetName); } - @Test + @TestTemplate public void testRegisterTable() throws NoSuchTableException, ParseException { long numRows = 1000; @@ -70,16 +69,16 @@ public void testRegisterTable() throws NoSuchTableException, ParseException { List result = sql("CALL %s.system.register_table('%s', '%s')", catalogName, targetName, metadataJson); - Assert.assertEquals("Current Snapshot is not correct", currentSnapshotId, result.get(0)[0]); + assertThat(result.get(0)[0]).as("Current Snapshot is not correct").isEqualTo(currentSnapshotId); List original = sql("SELECT * FROM %s", tableName); List registered = sql("SELECT * FROM %s", targetName); assertEquals("Registered table rows should match original table rows", original, registered); - Assert.assertEquals( - "Should have the right row count in the procedure result", numRows, result.get(0)[1]); - Assert.assertEquals( - "Should have the right datafile count in the procedure result", - originalFileCount, - result.get(0)[2]); + assertThat(result.get(0)[1]) + .as("Should have the right row count in the procedure result") + .isEqualTo(numRows); + assertThat(result.get(0)[2]) + .as("Should have the right datafile count in the procedure result") + .isEqualTo(originalFileCount); } } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 367d95d8f619..f0d1bdb82fca 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -18,40 +18,35 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.io.IOException; import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestSnapshotTableProcedure extends SparkExtensionsTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotTableProcedure extends ExtensionsTestBase { private static final String sourceName = "spark_catalog.default.source"; // Currently we can only Snapshot only out of the Spark Session Catalog - public TestSnapshotTableProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s PURGE", sourceName); } - @Test + @TestTemplate public void testSnapshot() throws IOException { - String location = temp.newFolder().toString(); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -59,11 +54,13 @@ public void testSnapshot() throws IOException { Object result = scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); Table createdTable = validationCatalog.loadTable(tableIdent); String tableLocation = createdTable.location(); - Assert.assertNotEquals("Table should not have the original location", location, tableLocation); + assertThat(tableLocation) + .as("Table should not have the original location") + .isNotEqualTo(location); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -73,9 +70,9 @@ public void testSnapshot() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSnapshotWithProperties() throws IOException { - String location = temp.newFolder().toString(); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -85,15 +82,17 @@ public void testSnapshotWithProperties() throws IOException { "CALL %s.system.snapshot(source_table => '%s', table => '%s', properties => map('foo','bar'))", catalogName, sourceName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); Table createdTable = validationCatalog.loadTable(tableIdent); String tableLocation = createdTable.location(); - Assert.assertNotEquals("Table should not have the original location", location, tableLocation); + assertThat(tableLocation) + .as("Table should not have the original location") + .isNotEqualTo(location); Map props = createdTable.properties(); - Assert.assertEquals("Should have extra property set", "bar", props.get("foo")); + assertThat(props.get("foo")).as("Should have extra property set").isEqualTo("bar"); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -103,13 +102,13 @@ public void testSnapshotWithProperties() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSnapshotWithAlternateLocation() throws IOException { - Assume.assumeTrue( - "No Snapshoting with Alternate locations with Hadoop Catalogs", - !catalogName.contains("hadoop")); - String location = temp.newFolder().toString(); - String snapshotLocation = temp.newFolder().toString(); + assumeThat(catalogName) + .as("No Snapshoting with Alternate locations with Hadoop Catalogs") + .doesNotContain("hadoop"); + String location = temp.toFile().toString(); + String snapshotLocation = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -120,11 +119,12 @@ public void testSnapshotWithAlternateLocation() throws IOException { catalogName, sourceName, tableName, snapshotLocation) .get(0); - Assert.assertEquals("Should have added one file", 1L, result[0]); + assertThat(result[0]).as("Should have added one file").isEqualTo(1L); String storageLocation = validationCatalog.loadTable(tableIdent).location(); - Assert.assertEquals( - "Snapshot should be made at specified location", snapshotLocation, storageLocation); + assertThat(storageLocation) + .as("Snapshot should be made at specified location") + .isEqualTo(snapshotLocation); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -134,9 +134,9 @@ public void testSnapshotWithAlternateLocation() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testDropTable() throws IOException { - String location = temp.newFolder().toString(); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -144,7 +144,7 @@ public void testDropTable() throws IOException { Object result = scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); assertEquals( "Should have expected rows", @@ -159,9 +159,9 @@ public void testDropTable() throws IOException { sql("SELECT * FROM %s", sourceName)); } - @Test + @TestTemplate public void testSnapshotWithConflictingProps() throws IOException { - String location = temp.newFolder().toString(); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -174,7 +174,7 @@ public void testSnapshotWithConflictingProps() throws IOException { + "table => '%s'," + "properties => map('%s', 'true', 'snapshot', 'false'))", catalogName, sourceName, tableName, TableProperties.GC_ENABLED); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); assertEquals( "Should have expected rows", @@ -183,14 +183,15 @@ public void testSnapshotWithConflictingProps() throws IOException { Table table = validationCatalog.loadTable(tableIdent); Map props = table.properties(); - Assert.assertEquals("Should override user value", "true", props.get("snapshot")); - Assert.assertEquals( - "Should override user value", "false", props.get(TableProperties.GC_ENABLED)); + assertThat(props.get("snapshot")).as("Should override user value").isEqualTo("true"); + assertThat(props.get(TableProperties.GC_ENABLED)) + .as("Should override user value") + .isEqualTo("false"); } - @Test + @TestTemplate public void testInvalidSnapshotsCases() throws IOException { - String location = temp.newFolder().toString(); + String location = temp.toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); From 5f20957777ad75e89138c2e1446626baca0e2866 Mon Sep 17 00:00:00 2001 From: Tomohiro Tanaka Date: Wed, 21 Feb 2024 20:38:57 +0900 Subject: [PATCH 2/6] Update the remaining procedures with JUnit5 and AssertJ --- .../spark/extensions/TestChangelogTable.java | 61 +++++---- .../TestCherrypickSnapshotProcedure.java | 24 ++-- .../TestCreateChangelogViewProcedure.java | 59 ++++----- .../TestExpireSnapshotsProcedure.java | 123 ++++++++---------- .../TestFastForwardBranchProcedure.java | 27 ++-- .../TestPublishChangesProcedure.java | 24 ++-- .../TestRemoveOrphanFilesProcedure.java | 107 +++++++-------- .../TestRewriteDataFilesProcedure.java | 96 +++++++------- .../TestRewriteManifestsProcedure.java | 111 +++++++++------- .../TestRewritePositionDeleteFiles.java | 52 ++++---- ...stRewritePositionDeleteFilesProcedure.java | 59 +++++---- .../TestRollbackToSnapshotProcedure.java | 37 +++--- .../TestRollbackToTimestampProcedure.java | 31 ++--- .../TestSetCurrentSnapshotProcedure.java | 35 +++-- 14 files changed, 394 insertions(+), 452 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java index fe44fa3dec52..daf45626753b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java @@ -21,59 +21,56 @@ import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; +import static org.assertj.core.api.Assertions.assertThat; import java.util.List; -import java.util.Map; import org.apache.iceberg.DataOperations; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.source.SparkChangelogTable; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Row; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestChangelogTable extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestChangelogTable extends ExtensionsTestBase { - @Parameters(name = "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, formatVersion = {3}") public static Object[][] parameters() { return new Object[][] { { - 1, SparkCatalogConfig.SPARK.catalogName(), SparkCatalogConfig.SPARK.implementation(), - SparkCatalogConfig.SPARK.properties() + SparkCatalogConfig.SPARK.properties(), + 1 }, { - 2, SparkCatalogConfig.HIVE.catalogName(), SparkCatalogConfig.HIVE.implementation(), - SparkCatalogConfig.HIVE.properties() + SparkCatalogConfig.HIVE.properties(), + 2 } }; } - private final int formatVersion; + @Parameter(index = 3) + private int formatVersion; - public TestChangelogTable( - int formatVersion, String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - this.formatVersion = formatVersion; - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testDataFilters() { createTableWithDefaultRows(); @@ -97,7 +94,7 @@ public void testDataFilters() { sql("SELECT * FROM %s.changes WHERE id = 3 ORDER BY _change_ordinal, id", tableName)); } - @Test + @TestTemplate public void testOverwrites() { createTableWithDefaultRows(); @@ -119,7 +116,7 @@ public void testOverwrites() { changelogRecords(snap2, snap3)); } - @Test + @TestTemplate public void testQueryWithTimeRange() { createTable(); @@ -189,7 +186,7 @@ public void testQueryWithTimeRange() { changelogRecords(rightAfterSnap2, snap3.timestampMillis() - 1)); } - @Test + @TestTemplate public void testTimeRangeValidation() { createTableWithDefaultRows(); @@ -207,7 +204,7 @@ public void testTimeRangeValidation() { .hasMessage("Cannot set start-timestamp to be greater than end-timestamp for changelogs"); } - @Test + @TestTemplate public void testMetadataDeletes() { createTableWithDefaultRows(); @@ -220,7 +217,7 @@ public void testMetadataDeletes() { table.refresh(); Snapshot snap3 = table.currentSnapshot(); - Assert.assertEquals("Operation must match", DataOperations.DELETE, snap3.operation()); + assertThat(snap3.operation()).as("Operation must match").isEqualTo(DataOperations.DELETE); assertEquals( "Rows should match", @@ -228,7 +225,7 @@ public void testMetadataDeletes() { changelogRecords(snap2, snap3)); } - @Test + @TestTemplate public void testExistingEntriesInNewDataManifestsAreIgnored() { sql( "CREATE TABLE %s (id INT, data STRING) " @@ -252,7 +249,7 @@ public void testExistingEntriesInNewDataManifestsAreIgnored() { table.refresh(); Snapshot snap2 = table.currentSnapshot(); - Assert.assertEquals("Manifest number must match", 1, snap2.dataManifests(table.io()).size()); + assertThat(snap2.dataManifests(table.io())).as("Manifest number must match").hasSize(1); assertEquals( "Rows should match", @@ -260,14 +257,14 @@ public void testExistingEntriesInNewDataManifestsAreIgnored() { changelogRecords(snap1, snap2)); } - @Test + @TestTemplate public void testManifestRewritesAreIgnored() { createTableWithDefaultRows(); sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Num snapshots must match", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Num snapshots must match").hasSize(3); assertEquals( "Should have expected rows", @@ -275,7 +272,7 @@ public void testManifestRewritesAreIgnored() { sql("SELECT id, _change_type FROM %s.changes ORDER BY id", tableName)); } - @Test + @TestTemplate public void testMetadataColumns() { createTableWithDefaultRows(); List rows = @@ -284,7 +281,7 @@ public void testMetadataColumns() { tableName); String file1 = rows.get(0)[1].toString(); - Assert.assertTrue(file1.startsWith("file:/")); + assertThat(file1).startsWith("file:/"); String file2 = rows.get(1)[1].toString(); assertEquals( diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java index c9af319f5c35..794d77980c74 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java @@ -21,7 +21,6 @@ import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import java.util.List; -import java.util.Map; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -32,22 +31,17 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestCherrypickSnapshotProcedure extends SparkExtensionsTestBase { +public class TestCherrypickSnapshotProcedure extends ExtensionsTestBase { - public TestCherrypickSnapshotProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testCherrypickSnapshotUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -84,7 +78,7 @@ public void testCherrypickSnapshotUsingPositionalArgs() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testCherrypickSnapshotUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -121,7 +115,7 @@ public void testCherrypickSnapshotUsingNamedArgs() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testCherrypickSnapshotRefreshesRelationCache() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -157,7 +151,7 @@ public void testCherrypickSnapshotRefreshesRelationCache() { sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testCherrypickInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -167,7 +161,7 @@ public void testCherrypickInvalidSnapshot() { .hasMessage("Cannot cherry-pick unknown snapshot ID: -1"); } - @Test + @TestTemplate public void testInvalidCherrypickSnapshotCases() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java index 9aa4bd3d7c8c..e12b4d4208a7 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java @@ -18,31 +18,28 @@ */ package org.apache.iceberg.spark.extensions; -import static org.junit.Assert.assertThrows; +import static org.assertj.core.api.Assertions.assertThat; import java.util.List; -import java.util.Map; import org.apache.iceberg.ChangelogOperation; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.SparkReadOptions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestCreateChangelogViewProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestCreateChangelogViewProcedure extends ExtensionsTestBase { private static final String DELETE = ChangelogOperation.DELETE.name(); private static final String INSERT = ChangelogOperation.INSERT.name(); private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name(); private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name(); - public TestCreateChangelogViewProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); } @@ -62,7 +59,7 @@ private void createTableWithIdentifierField() { sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); } - @Test + @TestTemplate public void testCustomizedViewName() { createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); @@ -92,10 +89,10 @@ public void testCustomizedViewName() { "cdc_view"); long rowCount = sql("select * from %s", "cdc_view").stream().count(); - Assert.assertEquals(2, rowCount); + assertThat(rowCount).isEqualTo(2); } - @Test + @TestTemplate public void testNoSnapshotIdInput() { createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); @@ -126,7 +123,7 @@ public void testNoSnapshotIdInput() { sql("select * from %s order by _change_ordinal, id", viewName)); } - @Test + @TestTemplate public void testTimestampsBasedQuery() { createTableWithTwoColumns(); long beginning = System.currentTimeMillis(); @@ -186,7 +183,7 @@ public void testTimestampsBasedQuery() { sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); } - @Test + @TestTemplate public void testUpdate() { createTableWithTwoColumns(); sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); @@ -217,7 +214,7 @@ public void testUpdate() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testUpdateWithIdentifierField() { createTableWithIdentifierField(); @@ -245,7 +242,7 @@ public void testUpdateWithIdentifierField() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testUpdateWithFilter() { createTableWithTwoColumns(); sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); @@ -277,7 +274,7 @@ public void testUpdateWithFilter() { sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testUpdateWithMultipleIdentifierColumns() { createTableWithThreeColumns(); @@ -309,7 +306,7 @@ public void testUpdateWithMultipleIdentifierColumns() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testRemoveCarryOvers() { createTableWithThreeColumns(); @@ -343,7 +340,7 @@ public void testRemoveCarryOvers() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testRemoveCarryOversWithoutUpdatedRows() { createTableWithThreeColumns(); @@ -375,7 +372,7 @@ public void testRemoveCarryOversWithoutUpdatedRows() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testNetChangesWithRemoveCarryOvers() { // partitioned by id createTableWithThreeColumns(); @@ -428,15 +425,15 @@ public void testNetChangesWithRemoveCarryOvers() { sql("select * from %s order by _change_ordinal, data", viewName)); } - @Test + @TestTemplate public void testNetChangesWithComputeUpdates() { createTableWithTwoColumns(); - assertThrows( - "Should fail because net_changes is not supported with computing updates", - IllegalArgumentException.class, - () -> - sql( - "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'), net_changes => true)", - catalogName, tableName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'), net_changes => true)", + catalogName, tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Not support net changes with update images"); } } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 7dacce5487d6..5df55e55aee2 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -27,7 +28,6 @@ import java.sql.Timestamp; import java.time.Instant; import java.util.List; -import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -43,7 +43,6 @@ import org.apache.iceberg.puffin.Puffin; import org.apache.iceberg.puffin.PuffinWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkCatalog; import org.apache.iceberg.spark.data.TestHelpers; @@ -52,23 +51,17 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestExpireSnapshotsProcedure extends SparkExtensionsTestBase { +public class TestExpireSnapshotsProcedure extends ExtensionsTestBase { - public TestExpireSnapshotsProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testExpireSnapshotsInEmptyTable() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -77,7 +70,7 @@ public void testExpireSnapshotsInEmptyTable() { "Should not delete any files", ImmutableList.of(row(0L, 0L, 0L, 0L, 0L, 0L)), output); } - @Test + @TestTemplate public void testExpireSnapshotsUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -95,7 +88,7 @@ public void testExpireSnapshotsUsingPositionalArgs() { Timestamp secondSnapshotTimestamp = Timestamp.from(Instant.ofEpochMilli(secondSnapshot.timestampMillis())); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); // expire without retainLast param List output1 = @@ -107,7 +100,7 @@ public void testExpireSnapshotsUsingPositionalArgs() { table.refresh(); - Assert.assertEquals("Should expire one snapshot", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should expire one snapshot").hasSize(1); sql("INSERT OVERWRITE %s VALUES (3, 'c')", tableName); sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName); @@ -122,7 +115,7 @@ public void testExpireSnapshotsUsingPositionalArgs() { Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); - Assert.assertEquals("Should be 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 3 snapshots").hasSize(3); // expire with retainLast param List output = @@ -133,7 +126,7 @@ public void testExpireSnapshotsUsingPositionalArgs() { "Procedure output must match", ImmutableList.of(row(2L, 0L, 0L, 2L, 1L, 0L)), output); } - @Test + @TestTemplate public void testExpireSnapshotUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -142,7 +135,7 @@ public void testExpireSnapshotUsingNamedArgs() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -156,7 +149,7 @@ public void testExpireSnapshotUsingNamedArgs() { "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output); } - @Test + @TestTemplate public void testExpireSnapshotsGCDisabled() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -168,7 +161,7 @@ public void testExpireSnapshotsGCDisabled() { .hasMessageStartingWith("Cannot expire snapshots: GC is disabled"); } - @Test + @TestTemplate public void testInvalidExpireSnapshotsCases() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) @@ -194,16 +187,12 @@ public void testInvalidExpireSnapshotsCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testResolvingTableInAnotherCatalog() throws IOException { String anotherCatalog = "another_" + catalogName; spark.conf().set("spark.sql.catalog." + anotherCatalog, SparkCatalog.class.getName()); spark.conf().set("spark.sql.catalog." + anotherCatalog + ".type", "hadoop"); - spark - .conf() - .set( - "spark.sql.catalog." + anotherCatalog + ".warehouse", - "file:" + temp.newFolder().toString()); + spark.conf().set("spark.sql.catalog." + anotherCatalog + ".warehouse", "file:" + temp.toFile()); sql( "CREATE TABLE %s.%s (id bigint NOT NULL, data string) USING iceberg", @@ -218,7 +207,7 @@ public void testResolvingTableInAnotherCatalog() throws IOException { .hasMessageStartingWith("Cannot run procedure in catalog"); } - @Test + @TestTemplate public void testConcurrentExpireSnapshots() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -241,7 +230,7 @@ public void testConcurrentExpireSnapshots() { output); } - @Test + @TestTemplate public void testConcurrentExpireSnapshotsWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -262,7 +251,7 @@ public void testConcurrentExpireSnapshotsWithInvalidInput() { .hasMessage("max_concurrent_deletes should have value > 0, value: -1"); } - @Test + @TestTemplate public void testExpireDeleteFiles() throws Exception { sql( "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" @@ -284,9 +273,8 @@ public void testExpireDeleteFiles() throws Exception { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); - Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteManifests(table)).as("Should have 1 delete manifest").hasSize(1); + assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); Path deleteFilePath = new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); @@ -306,13 +294,14 @@ public void testExpireDeleteFiles() throws Exception { sql("INSERT INTO TABLE %s VALUES (6, 'f')", tableName); // this txn removes the file reference table.refresh(); - Assert.assertEquals( - "Should have no delete manifests", 0, TestHelpers.deleteManifests(table).size()); - Assert.assertEquals("Should have no delete files", 0, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteManifests(table)).as("Should have no delete manifests").hasSize(0); + assertThat(TestHelpers.deleteFiles(table)).as("Should have no delete files").hasSize(0); FileSystem localFs = FileSystem.getLocal(new Configuration()); - Assert.assertTrue("Delete manifest should still exist", localFs.exists(deleteManifestPath)); - Assert.assertTrue("Delete file should still exist", localFs.exists(deleteFilePath)); + assertThat(localFs.exists(deleteManifestPath)) + .as("Delete manifest should still exist") + .isTrue(); + assertThat(localFs.exists(deleteFilePath)).as("Delete file should still exist").isTrue(); Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); List output = @@ -324,11 +313,13 @@ public void testExpireDeleteFiles() throws Exception { "Should deleted 1 data and pos delete file and 4 manifests and lists (one for each txn)", ImmutableList.of(row(1L, 1L, 0L, 4L, 4L, 0L)), output); - Assert.assertFalse("Delete manifest should be removed", localFs.exists(deleteManifestPath)); - Assert.assertFalse("Delete file should be removed", localFs.exists(deleteFilePath)); + assertThat(localFs.exists(deleteManifestPath)) + .as("Delete manifest should be removed") + .isFalse(); + assertThat(localFs.exists(deleteFilePath)).as("Delete file should be removed").isFalse(); } - @Test + @TestTemplate public void testExpireSnapshotWithStreamResultsEnabled() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -337,7 +328,7 @@ public void testExpireSnapshotWithStreamResultsEnabled() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -354,7 +345,7 @@ public void testExpireSnapshotWithStreamResultsEnabled() { "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output); } - @Test + @TestTemplate public void testExpireSnapshotsWithSnapshotId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -363,7 +354,7 @@ public void testExpireSnapshotsWithSnapshotId() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); // Expiring the snapshot specified by snapshot_id should keep only a single snapshot. long firstSnapshotId = table.currentSnapshot().parentId(); @@ -373,16 +364,14 @@ public void testExpireSnapshotsWithSnapshotId() { // There should only be one single snapshot left. table.refresh(); - Assert.assertEquals("Should be 1 snapshots", 1, Iterables.size(table.snapshots())); - Assert.assertEquals( - "Snapshot ID should not be present", - 0, - Iterables.size( - Iterables.filter( - table.snapshots(), snapshot -> snapshot.snapshotId() == firstSnapshotId))); + assertThat(table.snapshots()).as("Should be 1 snapshots").hasSize(1); + assertThat(table.snapshots()) + .as("Snapshot ID should not be present") + .filteredOn(snapshot -> snapshot.snapshotId() == firstSnapshotId) + .hasSize(0); } - @Test + @TestTemplate public void testExpireSnapshotShouldFailForCurrentSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -390,7 +379,7 @@ public void testExpireSnapshotShouldFailForCurrentSnapshot() { sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); Assertions.assertThatThrownBy( () -> @@ -406,7 +395,7 @@ public void testExpireSnapshotShouldFailForCurrentSnapshot() { .hasMessageStartingWith("Cannot expire"); } - @Test + @TestTemplate public void testExpireSnapshotsProcedureWorksWithSqlComments() { // Ensure that systems such as dbt, that inject comments into the generated SQL files, will // work with Iceberg-specific DDL @@ -417,7 +406,7 @@ public void testExpireSnapshotsProcedureWorksWithSqlComments() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -436,10 +425,10 @@ public void testExpireSnapshotsProcedureWorksWithSqlComments() { table.refresh(); - Assert.assertEquals("Should be 1 snapshot remaining", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 1 snapshot remaining").hasSize(1); } - @Test + @TestTemplate public void testExpireSnapshotsWithStatisticFiles() throws Exception { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (10, 'abc')", tableName); @@ -471,26 +460,26 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); + assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); table.refresh(); - Assertions.assertThat(table.statisticsFiles()) + assertThat(table.statisticsFiles()) .as( "Statistics file entry in TableMetadata should be present only for the snapshot %s", statisticsFile2.snapshotId()) .extracting(StatisticsFile::snapshotId) .containsExactly(statisticsFile2.snapshotId()); - Assertions.assertThat(new File(statsFileLocation1)) + assertThat(new File(statsFileLocation1)) .as("Statistics file should not exist for snapshot %s", statisticsFile1.snapshotId()) .doesNotExist(); - Assertions.assertThat(new File(statsFileLocation2)) + assertThat(new File(statsFileLocation2)) .as("Statistics file should exist for snapshot %s", statisticsFile2.snapshotId()) .exists(); } - @Test + @TestTemplate public void testExpireSnapshotsWithPartitionStatisticFiles() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (10, 'abc')", tableName); @@ -516,25 +505,23 @@ public void testExpireSnapshotsWithPartitionStatisticFiles() { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]) - .as("should be 1 deleted partition statistics file") - .isEqualTo(1L); + assertThat(output.get(0)[5]).as("should be 1 deleted partition statistics file").isEqualTo(1L); table.refresh(); - Assertions.assertThat(table.partitionStatisticsFiles()) + assertThat(table.partitionStatisticsFiles()) .as( "partition statistics file entry in TableMetadata should be present only for the snapshot %s", partitionStatisticsFile2.snapshotId()) .extracting(PartitionStatisticsFile::snapshotId) .containsExactly(partitionStatisticsFile2.snapshotId()); - Assertions.assertThat(new File(partitionStatsFileLocation1)) + assertThat(new File(partitionStatsFileLocation1)) .as( "partition statistics file should not exist for snapshot %s", partitionStatisticsFile1.snapshotId()) .doesNotExist(); - Assertions.assertThat(new File(partitionStatsFileLocation2)) + assertThat(new File(partitionStatsFileLocation2)) .as( "partition statistics file should exist for snapshot %s", partitionStatisticsFile2.snapshotId()) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java index 0c99c3e07f6b..fd8ee7d91bdc 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java @@ -23,7 +23,6 @@ import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotRef; @@ -31,21 +30,17 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestFastForwardBranchProcedure extends SparkExtensionsTestBase { - public TestFastForwardBranchProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +public class TestFastForwardBranchProcedure extends ExtensionsTestBase { - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testFastForwardBranchUsingPositionalArgs() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -96,7 +91,7 @@ public void testFastForwardBranchUsingPositionalArgs() { sql("SELECT * FROM %s order by id", tableName)); } - @Test + @TestTemplate public void testFastForwardBranchUsingNamedArgs() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -129,7 +124,7 @@ public void testFastForwardBranchUsingNamedArgs() { sql("SELECT * FROM %s order by id", tableName)); } - @Test + @TestTemplate public void testFastForwardWhenTargetIsNotAncestorFails() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -163,7 +158,7 @@ public void testFastForwardWhenTargetIsNotAncestorFails() { .hasMessage("Cannot fast-forward: main is not an ancestor of testBranch"); } - @Test + @TestTemplate public void testInvalidFastForwardBranchCases() { assertThatThrownBy( () -> @@ -189,7 +184,7 @@ public void testInvalidFastForwardBranchCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testFastForwardNonExistingToRefFails() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); assertThatThrownBy( @@ -201,7 +196,7 @@ public void testFastForwardNonExistingToRefFails() { .hasMessage("Ref does not exist: non_existing_branch"); } - @Test + @TestTemplate public void testFastForwardNonMain() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -231,7 +226,7 @@ public void testFastForwardNonMain() { .containsExactly(row(branch1, branch1Snapshot.snapshotId(), branch2Snapshot.snapshotId())); } - @Test + @TestTemplate public void testFastForwardNonExistingFromMainCreatesBranch() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); String branch1 = "branch1"; diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java index 94c4af3ad56d..7121c59df541 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java @@ -21,7 +21,6 @@ import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import java.util.List; -import java.util.Map; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -32,22 +31,17 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestPublishChangesProcedure extends SparkExtensionsTestBase { +public class TestPublishChangesProcedure extends ExtensionsTestBase { - public TestPublishChangesProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testApplyWapChangesUsingPositionalArgs() { String wapId = "wap_id_1"; sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -83,7 +77,7 @@ public void testApplyWapChangesUsingPositionalArgs() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testApplyWapChangesUsingNamedArgs() { String wapId = "wap_id_1"; sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -121,7 +115,7 @@ public void testApplyWapChangesUsingNamedArgs() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testApplyWapChangesRefreshesRelationCache() { String wapId = "wap_id_1"; sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -153,7 +147,7 @@ public void testApplyWapChangesRefreshesRelationCache() { sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testApplyInvalidWapId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -163,7 +157,7 @@ public void testApplyInvalidWapId() { .hasMessage("Cannot apply unknown WAP ID 'not_valid'"); } - @Test + @TestTemplate public void testInvalidApplyWapChangesCases() { Assertions.assertThatThrownBy( () -> diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 80b515d34417..74dbccc17fc9 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -29,7 +30,6 @@ import java.sql.Timestamp; import java.time.Instant; import java.util.List; -import java.util.Map; import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -65,28 +65,18 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestRemoveOrphanFilesProcedure extends SparkExtensionsTestBase { +public class TestRemoveOrphanFilesProcedure extends ExtensionsTestBase { - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - public TestRemoveOrphanFilesProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s PURGE", tableName); sql("DROP TABLE IF EXISTS p PURGE"); } - @Test + @TestTemplate public void testRemoveOrphanFilesInEmptyTable() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -97,7 +87,7 @@ public void testRemoveOrphanFilesInEmptyTable() { assertEquals("Should have no rows", ImmutableList.of(), sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testRemoveOrphanFilesInDataFolder() throws IOException { if (catalogName.equals("testhadoop")) { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -106,7 +96,7 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.newFolder()); + tableName, temp.toFile()); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -143,7 +133,7 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be orphan files in the data folder", 1, output2.size()); + assertThat(output2).as("Should be orphan files in the data folder").hasSize(1); // the previous call should have deleted all orphan files List output3 = @@ -152,7 +142,7 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be no more orphan files in the data folder", 0, output3.size()); + assertThat(output3).as("Should be no more orphan files in the data folder").hasSize(0); assertEquals( "Should have expected rows", @@ -160,7 +150,7 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRemoveOrphanFilesDryRun() throws IOException { if (catalogName.equals("testhadoop")) { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -169,7 +159,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.newFolder()); + tableName, temp.toFile()); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -194,7 +184,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { + "older_than => TIMESTAMP '%s'," + "dry_run => true)", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be one orphan files", 1, output1.size()); + assertThat(output1).as("Should be one orphan files").hasSize(1); // actually delete orphans List output2 = @@ -203,7 +193,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be one orphan files", 1, output2.size()); + assertThat(output2).as("Should be one orphan files").hasSize(1); // the previous call should have deleted all orphan files List output3 = @@ -212,7 +202,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be no more orphan files", 0, output3.size()); + assertThat(output3).as("Should be no more orphan files").hasSize(0); assertEquals( "Should have expected rows", @@ -220,7 +210,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRemoveOrphanFilesGCDisabled() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -236,7 +226,7 @@ public void testRemoveOrphanFilesGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, GC_ENABLED); } - @Test + @TestTemplate public void testRemoveOrphanFilesWap() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -255,7 +245,7 @@ public void testRemoveOrphanFilesWap() { assertEquals("Should be no orphan files", ImmutableList.of(), output); } - @Test + @TestTemplate public void testInvalidRemoveOrphanFilesCases() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)) @@ -281,7 +271,7 @@ public void testInvalidRemoveOrphanFilesCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testConcurrentRemoveOrphanFiles() throws IOException { if (catalogName.equals("testhadoop")) { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -290,7 +280,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.newFolder()); + tableName, temp.toFile()); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -321,7 +311,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { + "max_concurrent_deletes => %s," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, 4, currentTimestamp); - Assert.assertEquals("Should be orphan files in the data folder", 4, output.size()); + assertThat(output).as("Should be orphan files in the data folder").hasSize(4); // the previous call should have deleted all orphan files List output3 = @@ -331,7 +321,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { + "max_concurrent_deletes => %s," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, 4, currentTimestamp); - Assert.assertEquals("Should be no more orphan files in the data folder", 0, output3.size()); + assertThat(output3).as("Should be no more orphan files in the data folder").hasSize(0); assertEquals( "Should have expected rows", @@ -339,7 +329,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testConcurrentRemoveOrphanFilesWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -397,7 +387,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .hasMessage("Invalid last_modified column: StringType is not a timestamp"); } - @Test + @TestTemplate public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { sql( "CREATE TABLE %s (id int, data string) USING iceberg TBLPROPERTIES" @@ -418,9 +408,8 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { sql("DELETE FROM %s WHERE id=1", tableName); Table table = Spark3Util.loadIcebergTable(spark, tableName); - Assert.assertEquals( - "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); - Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteManifests(table)).as("Should have 1 delete manifest").hasSize(1); + assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); Path deleteFilePath = new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); @@ -436,20 +425,22 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be no orphan files", 0, output.size()); + assertThat(output).as("Should be no orphan files").hasSize(0); FileSystem localFs = FileSystem.getLocal(new Configuration()); - Assert.assertTrue("Delete manifest should still exist", localFs.exists(deleteManifestPath)); - Assert.assertTrue("Delete file should still exist", localFs.exists(deleteFilePath)); + assertThat(localFs.exists(deleteManifestPath)) + .as("Delete manifest should still exist") + .isTrue(); + assertThat(localFs.exists(deleteFilePath)).as("Delete file should still exist").isTrue(); records.remove(new SimpleRecord(1, "a")); Dataset resultDF = spark.read().format("iceberg").load(tableName); List actualRecords = resultDF.as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", records, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(records); } - @Test + @TestTemplate public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { sql( "CREATE TABLE %s USING iceberg " @@ -505,10 +496,10 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be no orphan files").isEmpty(); + assertThat(output).as("Should be no orphan files").isEmpty(); - Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); - Assertions.assertThat(statsLocation.length()) + assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); + assertThat(statsLocation.length()) .as("stats file length") .isEqualTo(statisticsFile.fileSizeInBytes()); @@ -522,14 +513,14 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(output)) + assertThat(output).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(output)) .as("Deleted files") .containsExactly(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 + @TestTemplate public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception { sql( "CREATE TABLE %s USING iceberg " @@ -555,11 +546,9 @@ public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be no orphan files").isEmpty(); + assertThat(output).as("Should be no orphan files").isEmpty(); - Assertions.assertThat(new File(partitionStatsLocation)) - .as("partition stats file should exist") - .exists(); + assertThat(new File(partitionStatsLocation)).as("partition stats file should exist").exists(); removePartitionStatsTxn(table, partitionStatisticsFile); @@ -569,11 +558,11 @@ public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(output)) + assertThat(output).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(output)) .as("Deleted files") .containsExactly("file:" + partitionStatsLocation); - Assertions.assertThat(new File(partitionStatsLocation)) + assertThat(new File(partitionStatsLocation)) .as("partition stats file should be deleted") .doesNotExist(); } @@ -598,7 +587,7 @@ private static void commitPartitionStatsTxn( transaction.commitTransaction(); } - @Test + @TestTemplate public void testRemoveOrphanFilesProcedureWithPrefixMode() throws NoSuchTableException, ParseException, IOException { if (catalogName.equals("testhadoop")) { @@ -606,7 +595,7 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() } else { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.newFolder().toURI().toString()); + tableName, temp.toFile().toURI().toString()); } Table table = Spark3Util.loadIcebergTable(spark, tableName); String location = table.location(); @@ -663,7 +652,7 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() + "equal_schemes => map('file1', 'file')," + "file_list_view => '%s')", catalogName, tableIdent, fileListViewName); - Assert.assertEquals(0, orphanFiles.size()); + assertThat(orphanFiles).isEmpty(); // Test with no equal schemes Assertions.assertThatThrownBy( diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 477aa2a1d958..b01438d39dfc 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -19,11 +19,13 @@ package org.apache.iceberg.spark.extensions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.IntStream; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; @@ -43,51 +45,47 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.internal.SQLConf; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestRewriteDataFilesProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteDataFilesProcedure extends ExtensionsTestBase { private static final String QUOTED_SPECIAL_CHARS_TABLE_NAME = "`table:with.special:chars`"; - public TestRewriteDataFilesProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @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"); } - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s", tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); } - @Test + @TestTemplate public void testZOrderSortExpression() { List order = ExtendedParser.parseSortOrder(spark, "c1, zorder(c2, c3)"); - Assert.assertEquals("Should parse 2 order fields", 2, order.size()); - Assert.assertEquals( - "First field should be a ref", "c1", ((NamedReference) order.get(0).term()).name()); - Assert.assertTrue("Second field should be zorder", order.get(1).term() instanceof Zorder); + assertThat(order).as("Should parse 2 order fields").hasSize(2); + assertThat(((NamedReference) order.get(0).term()).name()) + .as("First field should be a ref") + .isEqualTo("c1"); + assertThat(order.get(1).term()).as("Second field should be zorder").isInstanceOf(Zorder.class); } - @Test + @TestTemplate public void testRewriteDataFilesInEmptyTable() { createTable(); List output = sql("CALL %s.system.rewrite_data_files('%s')", catalogName, tableIdent); assertEquals("Procedure output must match", ImmutableList.of(row(0, 0, 0L, 0)), output); } - @Test + @TestTemplate public void testRewriteDataFilesOnPartitionTable() { createPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') @@ -111,7 +109,7 @@ public void testRewriteDataFilesOnPartitionTable() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesOnNonPartitionTable() { createTable(); // create 10 files under non-partitioned table @@ -135,7 +133,7 @@ public void testRewriteDataFilesOnNonPartitionTable() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithOptions() { createTable(); // create 10 files under non-partitioned table @@ -157,7 +155,7 @@ public void testRewriteDataFilesWithOptions() { assertEquals("Data should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithSortStrategy() { createTable(); // create 10 files under non-partitioned table @@ -185,7 +183,7 @@ public void testRewriteDataFilesWithSortStrategy() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithSortStrategyAndMultipleShufflePartitionsPerFile() { createTable(); insertData(10 /* file count */); @@ -220,7 +218,7 @@ public void testRewriteDataFilesWithSortStrategyAndMultipleShufflePartitionsPerF assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testRewriteDataFilesWithZOrder() { createTable(); // create 10 files under non-partitioned table @@ -261,7 +259,7 @@ public void testRewriteDataFilesWithZOrder() { assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testRewriteDataFilesWithZOrderAndMultipleShufflePartitionsPerFile() { createTable(); insertData(10 /* file count */); @@ -297,7 +295,7 @@ public void testRewriteDataFilesWithZOrderAndMultipleShufflePartitionsPerFile() assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testRewriteDataFilesWithFilter() { createTable(); // create 10 files under non-partitioned table @@ -325,7 +323,7 @@ public void testRewriteDataFilesWithFilter() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithDeterministicTrueFilter() { createTable(); // create 10 files under non-partitioned table @@ -349,7 +347,7 @@ public void testRewriteDataFilesWithDeterministicTrueFilter() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithDeterministicFalseFilter() { createTable(); // create 10 files under non-partitioned table @@ -368,7 +366,7 @@ public void testRewriteDataFilesWithDeterministicFalseFilter() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithFilterOnPartitionTable() { createPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') @@ -396,11 +394,11 @@ public void testRewriteDataFilesWithFilterOnPartitionTable() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithFilterOnOnBucketExpression() { // currently spark session catalog only resolve to v1 functions instead of desired v2 functions // https://github.com/apache/spark/blob/branch-3.4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L2070-L2083 - Assume.assumeFalse(catalogName.equals(SparkCatalogConfig.SPARK.catalogName())); + assumeThat(catalogName).isNotEqualTo(SparkCatalogConfig.SPARK.catalogName()); createBucketPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') insertData(10); @@ -428,7 +426,7 @@ public void testRewriteDataFilesWithFilterOnOnBucketExpression() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithInFilterOnPartitionTable() { createPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') @@ -456,7 +454,7 @@ public void testRewriteDataFilesWithInFilterOnPartitionTable() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithAllPossibleFilters() { createPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') @@ -523,11 +521,11 @@ public void testRewriteDataFilesWithAllPossibleFilters() { // " where => 'c2 like \"%s\"')", catalogName, tableIdent, "%car%"); } - @Test + @TestTemplate public void testRewriteDataFilesWithPossibleV2Filters() { // currently spark session catalog only resolve to v1 functions instead of desired v2 functions // https://github.com/apache/spark/blob/branch-3.4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L2070-L2083 - Assume.assumeFalse(catalogName.equals(SparkCatalogConfig.SPARK.catalogName())); + assumeThat(catalogName).isNotEqualTo(SparkCatalogConfig.SPARK.catalogName()); SystemFunctionPushDownHelper.createPartitionedTable(spark, tableName, "id"); sql( @@ -556,7 +554,7 @@ public void testRewriteDataFilesWithPossibleV2Filters() { catalogName, tableIdent, catalogName); } - @Test + @TestTemplate public void testRewriteDataFilesWithInvalidInputs() { createTable(); // create 2 files under non-partitioned table @@ -654,7 +652,7 @@ public void testRewriteDataFilesWithInvalidInputs() { "Cannot mix identity sort columns and a Zorder sort expression:" + " c1,zorder(c2,c3)"); } - @Test + @TestTemplate public void testInvalidCasesForRewriteDataFiles() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)) @@ -680,9 +678,9 @@ public void testInvalidCasesForRewriteDataFiles() { .hasMessage("Cannot handle an empty identifier for parameter 'table'"); } - @Test + @TestTemplate public void testBinPackTableWithSpecialChars() { - Assume.assumeTrue(catalogName.equals(SparkCatalogConfig.HADOOP.catalogName())); + assumeThat(catalogName).isEqualTo(SparkCatalogConfig.HADOOP.catalogName()); TableIdentifier identifier = TableIdentifier.of("default", QUOTED_SPECIAL_CHARS_TABLE_NAME.replaceAll("`", "")); @@ -712,12 +710,12 @@ public void testBinPackTableWithSpecialChars() { List actualRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); assertEquals("Data after compaction should not change", expectedRecords, actualRecords); - Assert.assertEquals("Table cache must be empty", 0, SparkTableCache.get().size()); + assertThat(SparkTableCache.get().size()).as("Table cache must be empty").isEqualTo(0); } - @Test + @TestTemplate public void testSortTableWithSpecialChars() { - Assume.assumeTrue(catalogName.equals(SparkCatalogConfig.HADOOP.catalogName())); + assumeThat(catalogName).isEqualTo(SparkCatalogConfig.HADOOP.catalogName()); TableIdentifier identifier = TableIdentifier.of("default", QUOTED_SPECIAL_CHARS_TABLE_NAME.replaceAll("`", "")); @@ -752,12 +750,12 @@ public void testSortTableWithSpecialChars() { List actualRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); assertEquals("Data after compaction should not change", expectedRecords, actualRecords); - Assert.assertEquals("Table cache must be empty", 0, SparkTableCache.get().size()); + assertThat(SparkTableCache.get().size()).as("Table cache must be empty").isEqualTo(0); } - @Test + @TestTemplate public void testZOrderTableWithSpecialChars() { - Assume.assumeTrue(catalogName.equals(SparkCatalogConfig.HADOOP.catalogName())); + assumeThat(catalogName).isEqualTo(SparkCatalogConfig.HADOOP.catalogName()); TableIdentifier identifier = TableIdentifier.of("default", QUOTED_SPECIAL_CHARS_TABLE_NAME.replaceAll("`", "")); @@ -792,10 +790,10 @@ public void testZOrderTableWithSpecialChars() { List actualRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); assertEquals("Data after compaction should not change", expectedRecords, actualRecords); - Assert.assertEquals("Table cache must be empty", 0, SparkTableCache.get().size()); + assertThat(SparkTableCache.get().size()).as("Table cache must be empty").isEqualTo(0); } - @Test + @TestTemplate public void testDefaultSortOrder() { createTable(); // add a default sort order for a table @@ -828,7 +826,7 @@ public void testDefaultSortOrder() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteWithUntranslatedOrUnconvertedFilter() { createTable(); Assertions.assertThatThrownBy( diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index b230a6912984..e527eadeb081 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -19,11 +19,12 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; import java.sql.Date; import java.sql.Timestamp; import java.util.List; -import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -32,30 +33,26 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestRewriteManifestsProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteManifestsProcedure extends ExtensionsTestBase { - public TestRewriteManifestsProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testRewriteManifestsInEmptyTable() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); assertEquals("Procedure output must match", ImmutableList.of(row(0, 0)), output); } - @Test + @TestTemplate public void testRewriteLargeManifests() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -64,8 +61,9 @@ public void testRewriteLargeManifests() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", tableName); @@ -74,11 +72,12 @@ public void testRewriteLargeManifests() { table.refresh(); - Assert.assertEquals( - "Must have 4 manifests", 4, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); } - @Test + @TestTemplate public void testRewriteManifestsNoOp() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -87,8 +86,9 @@ public void testRewriteManifestsNoOp() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); // should not rewrite any manifests for no-op (output of rewrite is same as before and after) @@ -96,11 +96,12 @@ public void testRewriteManifestsNoOp() { table.refresh(); - Assert.assertEquals( - "Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); } - @Test + @TestTemplate public void testRewriteLargeManifestsOnDatePartitionedTableWithJava8APIEnabled() { withSQLConf( ImmutableMap.of("spark.sql.datetime.java8API.enabled", "true"), @@ -126,8 +127,9 @@ public void testRewriteLargeManifestsOnDatePartitionedTableWithJava8APIEnabled() Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); sql( "ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", @@ -139,12 +141,13 @@ public void testRewriteLargeManifestsOnDatePartitionedTableWithJava8APIEnabled() table.refresh(); - Assert.assertEquals( - "Must have 4 manifests", 4, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); }); } - @Test + @TestTemplate public void testRewriteLargeManifestsOnTimestampPartitionedTableWithJava8APIEnabled() { withSQLConf( ImmutableMap.of("spark.sql.datetime.java8API.enabled", "true"), @@ -174,8 +177,9 @@ public void testRewriteLargeManifestsOnTimestampPartitionedTableWithJava8APIEnab Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); sql( "ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", @@ -187,12 +191,13 @@ public void testRewriteLargeManifestsOnTimestampPartitionedTableWithJava8APIEnab table.refresh(); - Assert.assertEquals( - "Must have 4 manifests", 4, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); }); } - @Test + @TestTemplate public void testRewriteSmallManifestsWithSnapshotIdInheritance() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -209,8 +214,9 @@ public void testRewriteSmallManifestsWithSnapshotIdInheritance() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 4 manifest", 4, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); List output = sql("CALL %s.system.rewrite_manifests(table => '%s')", catalogName, tableIdent); @@ -218,11 +224,12 @@ public void testRewriteSmallManifestsWithSnapshotIdInheritance() { table.refresh(); - Assert.assertEquals( - "Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); } - @Test + @TestTemplate public void testRewriteSmallManifestsWithoutCaching() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -233,8 +240,9 @@ public void testRewriteSmallManifestsWithoutCaching() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 2 manifest", 2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 2 manifest") + .hasSize(2); List output = sql( @@ -244,11 +252,12 @@ public void testRewriteSmallManifestsWithoutCaching() { table.refresh(); - Assert.assertEquals( - "Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); } - @Test + @TestTemplate public void testRewriteManifestsCaseInsensitiveArgs() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -259,8 +268,9 @@ public void testRewriteManifestsCaseInsensitiveArgs() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 2 manifest", 2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 2 manifests") + .hasSize(2); List output = sql( @@ -270,11 +280,12 @@ public void testRewriteManifestsCaseInsensitiveArgs() { table.refresh(); - Assert.assertEquals( - "Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); } - @Test + @TestTemplate public void testInvalidRewriteManifestsCases() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)) @@ -305,7 +316,7 @@ public void testInvalidRewriteManifestsCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testReplacePartitionField() { sql( "CREATE TABLE %s (id int, ts timestamp, day_of_ts date) USING iceberg PARTITIONED BY (day_of_ts)", @@ -339,7 +350,7 @@ public void testReplacePartitionField() { sql("SELECT * FROM %s WHERE ts < current_timestamp() order by 1 asc", tableName)); } - @Test + @TestTemplate public void testWriteManifestWithSpecId() { sql( "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java index 4c5082cabbb6..f7329e841800 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -32,6 +32,7 @@ import java.time.LocalDateTime; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.iceberg.ContentFile; @@ -42,6 +43,7 @@ import org.apache.iceberg.Files; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PositionDeletesScanTask; import org.apache.iceberg.RowDelta; import org.apache.iceberg.ScanTask; @@ -69,13 +71,10 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.types.StructType; -import org.junit.After; -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; -public class TestRewritePositionDeleteFiles extends SparkExtensionsTestBase { +public class TestRewritePositionDeleteFiles extends ExtensionsTestBase { private static final Map CATALOG_PROPS = ImmutableMap.of( @@ -89,8 +88,7 @@ public class TestRewritePositionDeleteFiles extends SparkExtensionsTestBase { private static final int DELETE_FILES_PER_PARTITION = 2; private static final int DELETE_FILE_SIZE = 10; - @Parameterized.Parameters( - name = "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}") + @Parameters(name = "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}") public static Object[][] parameters() { return new Object[][] { { @@ -101,19 +99,12 @@ public static Object[][] parameters() { }; } - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - public TestRewritePositionDeleteFiles( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void cleanup() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testDatePartition() throws Exception { createTable("date"); Date baseDate = Date.valueOf("2023-01-01"); @@ -121,14 +112,14 @@ public void testDatePartition() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testBooleanPartition() throws Exception { createTable("boolean"); insertData(i -> i % 2 == 0, 2); testDanglingDelete(2); } - @Test + @TestTemplate public void testTimestampPartition() throws Exception { createTable("timestamp"); Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); @@ -136,7 +127,7 @@ public void testTimestampPartition() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testTimestampNtz() throws Exception { createTable("timestamp_ntz"); LocalDateTime baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00").toLocalDateTime(); @@ -144,14 +135,14 @@ public void testTimestampNtz() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testBytePartition() throws Exception { createTable("byte"); insertData(i -> i); testDanglingDelete(); } - @Test + @TestTemplate public void testDecimalPartition() throws Exception { createTable("decimal(18, 10)"); BigDecimal baseDecimal = new BigDecimal("1.0"); @@ -159,35 +150,35 @@ public void testDecimalPartition() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testBinaryPartition() throws Exception { createTable("binary"); insertData(i -> java.nio.ByteBuffer.allocate(4).putInt(i).array()); testDanglingDelete(); } - @Test + @TestTemplate public void testCharPartition() throws Exception { createTable("char(10)"); insertData(Object::toString); testDanglingDelete(); } - @Test + @TestTemplate public void testVarcharPartition() throws Exception { createTable("varchar(10)"); insertData(Object::toString); testDanglingDelete(); } - @Test + @TestTemplate public void testIntPartition() throws Exception { createTable("int"); insertData(i -> i); testDanglingDelete(); } - @Test + @TestTemplate public void testDaysPartitionTransform() throws Exception { createTable("timestamp", PARTITION_COL, String.format("days(%s)", PARTITION_COL)); Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); @@ -195,14 +186,14 @@ public void testDaysPartitionTransform() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testNullTransform() throws Exception { createTable("int"); insertData(i -> i == 0 ? null : 1, 2); testDanglingDelete(2); } - @Test + @TestTemplate public void testPartitionColWithDot() throws Exception { String partitionColWithDot = "`partition.col`"; createTable("int", partitionColWithDot, partitionColWithDot); @@ -318,7 +309,8 @@ private void writePosDeletesForFiles(Table table, List files) throws I counter++; if (counter == deleteFileSize) { // Dump to file and reset variables - OutputFile output = Files.localOutput(temp.newFile()); + OutputFile output = + Files.localOutput(temp.resolve(UUID.randomUUID().toString()).toFile()); deleteFiles.add(writeDeleteFile(table, output, partition, deletes)); counter = 0; deletes.clear(); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index 5dde5d698ee4..3ef03ec98945 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -20,9 +20,11 @@ import static org.apache.iceberg.SnapshotSummary.ADDED_FILE_SIZE_PROP; import static org.apache.iceberg.SnapshotSummary.REMOVED_FILE_SIZE_PROP; +import static org.assertj.core.api.Assertions.assertThat; import java.util.List; import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -30,16 +32,12 @@ import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.Encoders; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestRewritePositionDeleteFilesProcedure extends SparkExtensionsTestBase { - - public TestRewritePositionDeleteFilesProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewritePositionDeleteFilesProcedure extends ExtensionsTestBase { private void createTable() throws Exception { createTable(false); @@ -79,12 +77,12 @@ private void createTable(boolean partitioned) throws Exception { .append(); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testExpireDeleteFilesAll() throws Exception { createTable(); @@ -92,7 +90,7 @@ public void testExpireDeleteFilesAll() throws Exception { sql("DELETE FROM %s WHERE id=2", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals(2, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(2); List output = sql( @@ -114,10 +112,10 @@ public void testExpireDeleteFilesAll() throws Exception { Long.valueOf(snapshotSummary.get(ADDED_FILE_SIZE_PROP)))), output); - Assert.assertEquals(1, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(1); } - @Test + @TestTemplate public void testExpireDeleteFilesNoOption() throws Exception { createTable(); @@ -128,7 +126,7 @@ public void testExpireDeleteFilesNoOption() throws Exception { sql("DELETE FROM %s WHERE id=5", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals(5, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(5); List output = sql( @@ -148,7 +146,7 @@ public void testExpireDeleteFilesNoOption() throws Exception { output); } - @Test + @TestTemplate public void testExpireDeleteFilesFilter() throws Exception { createTable(true); @@ -160,7 +158,7 @@ public void testExpireDeleteFilesFilter() throws Exception { sql("DELETE FROM %s WHERE id = 3 and data='h'", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals(6, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(6); List output = sql( @@ -184,26 +182,27 @@ public void testExpireDeleteFilesFilter() throws Exception { Long.valueOf(snapshotSummary.get(ADDED_FILE_SIZE_PROP)))), output); - Assert.assertEquals(4, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(4); } - @Test + @TestTemplate public void testInvalidOption() throws Exception { createTable(); - Assert.assertThrows( - "Cannot use options [foo], they are not supported by the action or the rewriter BIN-PACK", - IllegalArgumentException.class, - () -> - sql( - "CALL %s.system.rewrite_position_delete_files(" - + "table => '%s'," - + "options => map(" - + "'foo', 'bar'))", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_position_delete_files(" + + "table => '%s'," + + "options => map(" + + "'foo', 'bar'))", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot use options [foo], they are not supported by the action or the rewriter BIN-PACK"); } - @Test + @TestTemplate public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { createTable(); Assertions.assertThatThrownBy( diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java index 1b4cd2b4a3f6..ba833bcee35e 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java @@ -18,8 +18,10 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.util.List; -import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -29,23 +31,18 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assume; -import org.junit.Test; - -public class TestRollbackToSnapshotProcedure extends SparkExtensionsTestBase { - - public TestRollbackToSnapshotProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; - @After +@ExtendWith(ParameterizedTestExtension.class) +public class TestRollbackToSnapshotProcedure extends ExtensionsTestBase { + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testRollbackToSnapshotUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -80,7 +77,7 @@ public void testRollbackToSnapshotUsingPositionalArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToSnapshotUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -115,7 +112,7 @@ public void testRollbackToSnapshotUsingNamedArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToSnapshotRefreshesRelationCache() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -155,7 +152,7 @@ public void testRollbackToSnapshotRefreshesRelationCache() { sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testRollbackToSnapshotWithQuotedIdentifiers() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -200,9 +197,9 @@ public void testRollbackToSnapshotWithQuotedIdentifiers() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToSnapshotWithoutExplicitCatalog() { - Assume.assumeTrue("Working only with the session catalog", "spark_catalog".equals(catalogName)); + assumeThat(catalogName).as("Working only with the session catalog").isEqualTo("spark_catalog"); sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -236,7 +233,7 @@ public void testRollbackToSnapshotWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -246,7 +243,7 @@ public void testRollbackToInvalidSnapshot() { .hasMessage("Cannot roll back to unknown snapshot id: -1"); } - @Test + @TestTemplate public void testInvalidRollbackToSnapshotCases() { Assertions.assertThatThrownBy( () -> diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java index 80af6e7f5f4c..d9366c08dea3 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.time.LocalDateTime; import java.util.List; -import java.util.Map; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -29,23 +30,17 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assume; -import org.junit.Test; - -public class TestRollbackToTimestampProcedure extends SparkExtensionsTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; - public TestRollbackToTimestampProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +public class TestRollbackToTimestampProcedure extends ExtensionsTestBase { - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testRollbackToTimestampUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -83,7 +78,7 @@ public void testRollbackToTimestampUsingPositionalArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToTimestampUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -121,7 +116,7 @@ public void testRollbackToTimestampUsingNamedArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToTimestampRefreshesRelationCache() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -164,7 +159,7 @@ public void testRollbackToTimestampRefreshesRelationCache() { sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testRollbackToTimestampWithQuotedIdentifiers() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -210,9 +205,9 @@ public void testRollbackToTimestampWithQuotedIdentifiers() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToTimestampWithoutExplicitCatalog() { - Assume.assumeTrue("Working only with the session catalog", "spark_catalog".equals(catalogName)); + assumeThat(catalogName).as("Working only with the session catalog").isEqualTo("spark_catalog"); sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -251,7 +246,7 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java index e894ba4ff0ae..52278026f829 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java @@ -19,9 +19,10 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; -import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -30,23 +31,19 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestSetCurrentSnapshotProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestSetCurrentSnapshotProcedure extends ExtensionsTestBase { - public TestSetCurrentSnapshotProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testSetCurrentSnapshotUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -81,7 +78,7 @@ public void testSetCurrentSnapshotUsingPositionalArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSetCurrentSnapshotUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -116,7 +113,7 @@ public void testSetCurrentSnapshotUsingNamedArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSetCurrentSnapshotWap() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -149,9 +146,9 @@ public void testSetCurrentSnapshotWap() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void tesSetCurrentSnapshotWithoutExplicitCatalog() { - Assume.assumeTrue("Working only with the session catalog", "spark_catalog".equals(catalogName)); + assumeThat(catalogName).as("Working only with the session catalog").isEqualTo("spark_catalog"); sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -185,7 +182,7 @@ public void tesSetCurrentSnapshotWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSetCurrentSnapshotToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -195,7 +192,7 @@ public void testSetCurrentSnapshotToInvalidSnapshot() { .hasMessage("Cannot roll back to unknown snapshot id: -1"); } - @Test + @TestTemplate public void testInvalidRollbackToSnapshotCases() { Assertions.assertThatThrownBy( () -> @@ -248,7 +245,7 @@ public void testInvalidRollbackToSnapshotCases() { .hasMessage("Either snapshot_id or ref must be provided, not both"); } - @Test + @TestTemplate public void testSetCurrentSnapshotToRef() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); From 8f593a82effa15f2c73f6e5b28921d3db44ee6da Mon Sep 17 00:00:00 2001 From: Tomohiro Tanaka Date: Thu, 22 Feb 2024 23:04:27 +0900 Subject: [PATCH 3/6] Reflect the comments --- .../extensions/TestCallStatementParser.java | 22 ++++++------------- .../TestCreateChangelogViewProcedure.java | 4 ++-- .../extensions/TestMigrateTableProcedure.java | 6 ++--- .../TestRemoveOrphanFilesProcedure.java | 8 +++---- ...stRewritePositionDeleteFilesProcedure.java | 8 +++---- .../TestSnapshotTableProcedure.java | 10 ++++----- 6 files changed, 23 insertions(+), 35 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java index 14e78173e7f9..490127ab52f5 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java @@ -25,7 +25,6 @@ import java.sql.Timestamp; import java.time.Instant; import java.util.List; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.expressions.Expression; @@ -73,7 +72,7 @@ public static void stopSpark() { public void testCallWithPositionalArgs() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL c.n.func(1, '2', 3L, true, 1.0D, 9.0e1, 900e-1BD)"); - assertThat(seqAsJavaList(call.name())).hasSameElementsAs(ImmutableList.of("c", "n", "func")); + assertThat(seqAsJavaList(call.name())).containsExactly("c", "n", "func"); assertThat(seqAsJavaList(call.args())).hasSize(7); @@ -90,8 +89,7 @@ public void testCallWithPositionalArgs() throws ParseException { public void testCallWithNamedArgs() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, c2 => '2', c3 => true)"); - assertThat(seqAsJavaList(call.name())) - .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); assertThat(seqAsJavaList(call.args())).hasSize(3); @@ -103,8 +101,7 @@ public void testCallWithNamedArgs() throws ParseException { @Test public void testCallWithMixedArgs() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, '2')"); - assertThat(seqAsJavaList(call.name())) - .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); assertThat(seqAsJavaList(call.args())).hasSize(2); @@ -117,8 +114,7 @@ public void testCallWithTimestampArg() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(TIMESTAMP '2017-02-03T10:37:30.00Z')"); - assertThat(seqAsJavaList(call.name())) - .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); assertThat(seqAsJavaList(call.args())).hasSize(1); @@ -130,8 +126,7 @@ public void testCallWithTimestampArg() throws ParseException { public void testCallWithVarSubstitution() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func('${spark.extra.prop}')"); - assertThat(seqAsJavaList(call.name())) - .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); assertThat(seqAsJavaList(call.args())).hasSize(1); @@ -161,8 +156,7 @@ public void testCallStripsComments() throws ParseException { "CALL -- a line ending comment\n" + "cat.system.func('${spark.extra.prop}')"); for (String sqlText : callStatementsWithComments) { CallStatement call = (CallStatement) parser.parsePlan(sqlText); - assertThat(seqAsJavaList(call.name())) - .hasSameElementsAs(ImmutableList.of("cat", "system", "func")); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); assertThat(seqAsJavaList(call.args())).hasSize(1); @@ -201,9 +195,7 @@ private Literal toSparkLiteral(Object value, DataType dataType) { } private T checkCast(Object value, Class expectedClass) { - assertThat(expectedClass.isInstance(value)) - .as("Expected instance of " + expectedClass.getName()) - .isTrue(); + assertThat(value).isInstanceOf(expectedClass); return expectedClass.cast(value); } } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java index e12b4d4208a7..bc60759bd038 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.iceberg.ChangelogOperation; @@ -27,7 +28,6 @@ import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.SparkReadOptions; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -428,7 +428,7 @@ public void testNetChangesWithRemoveCarryOvers() { @TestTemplate public void testNetChangesWithComputeUpdates() { createTableWithTwoColumns(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'), net_changes => true)", diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index 7d98cc83c841..da3150363609 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -85,7 +85,7 @@ public void testMigrateWithOptions() throws IOException { Table createdTable = validationCatalog.loadTable(tableIdent); Map props = createdTable.properties(); - assertThat(props.get("foo")).as("Should have extra property set").isEqualTo("bar"); + assertThat(props).containsEntry("foo", "bar"); String tableLocation = createdTable.location().replace("file:", ""); assertThat(tableLocation).as("Table should have original location").isEqualTo(location); @@ -174,9 +174,7 @@ public void testMigrateWithConflictingProps() throws IOException { sql("SELECT * FROM %s", tableName)); Table table = validationCatalog.loadTable(tableIdent); - assertThat(table.properties().get("migrated")) - .as("Should override user value") - .isEqualTo("true"); + assertThat(table.properties()).containsEntry("migrated", "true"); } @TestTemplate diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 74dbccc17fc9..6ba8428723fd 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -498,10 +498,8 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { catalogName, tableIdent, currentTimestamp); assertThat(output).as("Should be no orphan files").isEmpty(); - assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); - assertThat(statsLocation.length()) - .as("stats file length") - .isEqualTo(statisticsFile.fileSizeInBytes()); + assertThat(statsLocation).exists(); + assertThat(statsLocation).hasSize(statisticsFile.fileSizeInBytes()); transaction = table.newTransaction(); transaction.updateStatistics().removeStatistics(statisticsFile.snapshotId()).commit(); @@ -517,7 +515,7 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { assertThat(Iterables.getOnlyElement(output)) .as("Deleted files") .containsExactly(statsLocation.toURI().toString()); - assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); + assertThat(statsLocation).doesNotExist(); } @TestTemplate diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index 3ef03ec98945..585db39a3c44 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.SnapshotSummary.ADDED_FILE_SIZE_PROP; import static org.apache.iceberg.SnapshotSummary.REMOVED_FILE_SIZE_PROP; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -31,7 +32,6 @@ import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.Encoders; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -189,7 +189,7 @@ public void testExpireDeleteFilesFilter() throws Exception { public void testInvalidOption() throws Exception { createTable(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_position_delete_files(" @@ -205,7 +205,7 @@ public void testInvalidOption() throws Exception { @TestTemplate public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { createTable(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'substr(encode(data, \"utf-8\"), 2) = \"fo\"')", @@ -213,7 +213,7 @@ public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot translate Spark expression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'substr(data, 2) = \"fo\"')", diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index f0d1bdb82fca..12b526313eda 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -92,7 +92,7 @@ public void testSnapshotWithProperties() throws IOException { .isNotEqualTo(location); Map props = createdTable.properties(); - assertThat(props.get("foo")).as("Should have extra property set").isEqualTo("bar"); + assertThat(props).as("Should have extra property set").containsEntry("foo", "bar"); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -108,7 +108,7 @@ public void testSnapshotWithAlternateLocation() throws IOException { .as("No Snapshoting with Alternate locations with Hadoop Catalogs") .doesNotContain("hadoop"); String location = temp.toFile().toString(); - String snapshotLocation = temp.toFile().toString(); + String snapshotLocation = temp.toFile() + "_snapshot"; sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -183,10 +183,10 @@ public void testSnapshotWithConflictingProps() throws IOException { Table table = validationCatalog.loadTable(tableIdent); Map props = table.properties(); - assertThat(props.get("snapshot")).as("Should override user value").isEqualTo("true"); - assertThat(props.get(TableProperties.GC_ENABLED)) + assertThat(props).as("Should override user value").containsEntry("snapshot", "true"); + assertThat(props) .as("Should override user value") - .isEqualTo("false"); + .containsEntry(TableProperties.GC_ENABLED, "false"); } @TestTemplate From a3756937d9f71a92c3231fe0e48d4d4415b12d44 Mon Sep 17 00:00:00 2001 From: Tomohiro Tanaka Date: Fri, 23 Feb 2024 22:13:29 +0900 Subject: [PATCH 4/6] Update temp directory creation with Files --- .../extensions/TestAddFilesProcedure.java | 6 ++++-- .../TestExpireSnapshotsProcedure.java | 7 ++++++- .../extensions/TestMigrateTableProcedure.java | 19 ++++++++++--------- .../TestRemoveOrphanFilesProcedure.java | 10 +++++----- .../TestSnapshotTableProcedure.java | 15 ++++++++------- .../spark/extensions/TestWriteAborts.java | 6 ++++-- 6 files changed, 37 insertions(+), 26 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 91c74a6ad435..9ad8bc476ff3 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -22,6 +22,8 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; +import java.io.IOException; +import java.nio.file.Files; import java.nio.file.Path; import java.util.List; import java.util.regex.Matcher; @@ -95,8 +97,8 @@ public static Object[][] parameters() { @TempDir private Path temp; @BeforeEach - public void setupTempDirs() { - fileTableDir = temp.toFile(); + public void setupTempDirs() throws IOException { + fileTableDir = Files.createTempDirectory(temp, "junit").toFile(); } @AfterEach diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 5df55e55aee2..46847a03ca7b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.nio.file.Files; import java.sql.Timestamp; import java.time.Instant; import java.util.List; @@ -192,7 +193,11 @@ public void testResolvingTableInAnotherCatalog() throws IOException { String anotherCatalog = "another_" + catalogName; spark.conf().set("spark.sql.catalog." + anotherCatalog, SparkCatalog.class.getName()); spark.conf().set("spark.sql.catalog." + anotherCatalog + ".type", "hadoop"); - spark.conf().set("spark.sql.catalog." + anotherCatalog + ".warehouse", "file:" + temp.toFile()); + spark + .conf() + .set( + "spark.sql.catalog." + anotherCatalog + ".warehouse", + Files.createTempDirectory(temp, "junit").toFile().toURI().toString()); sql( "CREATE TABLE %s.%s (id bigint NOT NULL, data string) USING iceberg", diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index da3150363609..26a888356ca6 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; +import java.nio.file.Files; import java.util.Map; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; @@ -44,7 +45,7 @@ public void removeTables() { @TestTemplate public void testMigrate() throws IOException { assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -71,7 +72,7 @@ public void testMigrate() throws IOException { @TestTemplate public void testMigrateWithOptions() throws IOException { assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -103,7 +104,7 @@ public void testMigrateWithOptions() throws IOException { @TestTemplate public void testMigrateWithDropBackup() throws IOException { assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -119,7 +120,7 @@ public void testMigrateWithDropBackup() throws IOException { @TestTemplate public void testMigrateWithBackupTableName() throws IOException { assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -140,7 +141,7 @@ public void testMigrateWithBackupTableName() throws IOException { public void testMigrateWithInvalidMetricsConfig() throws IOException { assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -158,7 +159,7 @@ public void testMigrateWithInvalidMetricsConfig() throws IOException { public void testMigrateWithConflictingProps() throws IOException { assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -196,7 +197,7 @@ public void testInvalidMigrateCases() { @TestTemplate public void testMigratePartitionWithSpecialCharacter() throws IOException { assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string, dt date) USING parquet " + "PARTITIONED BY (data, dt) LOCATION '%s'", @@ -213,7 +214,7 @@ public void testMigratePartitionWithSpecialCharacter() throws IOException { @TestTemplate public void testMigrateEmptyPartitionedTable() throws Exception { assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", tableName, location); @@ -224,7 +225,7 @@ public void testMigrateEmptyPartitionedTable() throws Exception { @TestTemplate public void testMigrateEmptyTable() throws Exception { assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 6ba8428723fd..9eb19613b361 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.extensions; +import static java.nio.file.Files.createTempDirectory; import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import static org.assertj.core.api.Assertions.assertThat; @@ -96,9 +97,8 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.toFile()); + tableName, createTempDirectory(temp, "junit")); } - sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); @@ -159,7 +159,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.toFile()); + tableName, createTempDirectory(temp, "junit")); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -280,7 +280,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.toFile()); + tableName, createTempDirectory(temp, "junit")); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -593,7 +593,7 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() } else { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.toFile().toURI().toString()); + tableName, createTempDirectory(temp, "junit")); } Table table = Spark3Util.loadIcebergTable(spark, tableName); String location = table.location(); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 12b526313eda..69bc35e9ea33 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; +import java.nio.file.Files; import java.util.Map; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; @@ -46,7 +47,7 @@ public void removeTables() { @TestTemplate public void testSnapshot() throws IOException { - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -72,7 +73,7 @@ public void testSnapshot() throws IOException { @TestTemplate public void testSnapshotWithProperties() throws IOException { - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -107,8 +108,8 @@ public void testSnapshotWithAlternateLocation() throws IOException { assumeThat(catalogName) .as("No Snapshoting with Alternate locations with Hadoop Catalogs") .doesNotContain("hadoop"); - String location = temp.toFile().toString(); - String snapshotLocation = temp.toFile() + "_snapshot"; + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + String snapshotLocation = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -136,7 +137,7 @@ public void testSnapshotWithAlternateLocation() throws IOException { @TestTemplate public void testDropTable() throws IOException { - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -161,7 +162,7 @@ public void testDropTable() throws IOException { @TestTemplate public void testSnapshotWithConflictingProps() throws IOException { - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -191,7 +192,7 @@ public void testSnapshotWithConflictingProps() throws IOException { @TestTemplate public void testInvalidSnapshotsCases() throws IOException { - String location = temp.toFile().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java index 256e654b7775..a40e66f5bf2b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -81,8 +83,8 @@ public void removeTables() { } @TestTemplate - public void testBatchAppend() throws Exception { - String dataLocation = temp.toFile().toString(); + public void testBatchAppend() throws IOException { + String dataLocation = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id INT, data STRING) " From e94bd2f81689f94def174469682e713902445fac Mon Sep 17 00:00:00 2001 From: Tomohiro Tanaka Date: Fri, 23 Feb 2024 23:11:13 +0900 Subject: [PATCH 5/6] Reflect the comments related to createTempDirectory --- .../iceberg/spark/extensions/TestAddFilesProcedure.java | 6 ++---- .../spark/extensions/TestRemoveOrphanFilesProcedure.java | 6 +++++- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 9ad8bc476ff3..91c74a6ad435 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -22,8 +22,6 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; -import java.io.IOException; -import java.nio.file.Files; import java.nio.file.Path; import java.util.List; import java.util.regex.Matcher; @@ -97,8 +95,8 @@ public static Object[][] parameters() { @TempDir private Path temp; @BeforeEach - public void setupTempDirs() throws IOException { - fileTableDir = Files.createTempDirectory(temp, "junit").toFile(); + public void setupTempDirs() { + fileTableDir = temp.toFile(); } @AfterEach diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 9eb19613b361..22757f911735 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark.extensions; -import static java.nio.file.Files.createTempDirectory; import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import static org.assertj.core.api.Assertions.assertThat; @@ -77,6 +76,11 @@ public void removeTable() { sql("DROP TABLE IF EXISTS p PURGE"); } + private java.nio.file.Path createTempDirectory(java.nio.file.Path temp, String prefix) + throws IOException { + return java.nio.file.Files.createTempDirectory(temp, prefix); + } + @TestTemplate public void testRemoveOrphanFilesInEmptyTable() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); From ae3d3e6db14dd01c586d2b313981c24c8d23161b Mon Sep 17 00:00:00 2001 From: Tomohiro Tanaka Date: Fri, 23 Feb 2024 23:57:05 +0900 Subject: [PATCH 6/6] Update createTempDirectory with specifying the full path --- .../extensions/TestRemoveOrphanFilesProcedure.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 22757f911735..5d48daa74f40 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -76,11 +76,6 @@ public void removeTable() { sql("DROP TABLE IF EXISTS p PURGE"); } - private java.nio.file.Path createTempDirectory(java.nio.file.Path temp, String prefix) - throws IOException { - return java.nio.file.Files.createTempDirectory(temp, prefix); - } - @TestTemplate public void testRemoveOrphanFilesInEmptyTable() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -101,7 +96,7 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, createTempDirectory(temp, "junit")); + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); @@ -163,7 +158,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, createTempDirectory(temp, "junit")); + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -284,7 +279,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, createTempDirectory(temp, "junit")); + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -597,7 +592,7 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() } else { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, createTempDirectory(temp, "junit")); + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); } Table table = Spark3Util.loadIcebergTable(spark, tableName); String location = table.location();