From a3d87e2368e56a33d7284f83c7ae64ea462c66d8 Mon Sep 17 00:00:00 2001 From: Chinmay Bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Sat, 13 Jan 2024 19:41:10 +0530 Subject: [PATCH] Spark 3.5: Migrate remaining tests in source directory to JUnit5 (#9380) --- .../spark/source/TestCompressionSettings.java | 106 ++++--- .../spark/source/TestFilteredScan.java | 135 ++++---- .../source/TestIdentityPartitionData.java | 117 ++++--- ...tMetadataTablesWithPartitionEvolution.java | 95 +++--- .../spark/source/TestPartitionPruning.java | 99 +++--- .../spark/source/TestPartitionValues.java | 117 +++---- .../source/TestPositionDeletesTable.java | 297 +++++++++++------- .../spark/source/TestReadProjection.java | 42 +-- .../TestRequiredDistributionAndOrdering.java | 37 +-- .../spark/source/TestRuntimeFiltering.java | 78 +++-- .../spark/source/TestSnapshotSelection.java | 178 ++++++----- .../TestSparkCatalogHadoopOverrides.java | 63 ++-- .../spark/source/TestSparkDataWrite.java | 207 ++++++------ .../source/TestSparkMetadataColumns.java | 82 ++--- .../spark/source/TestSparkReadProjection.java | 47 ++- .../TestSparkReaderWithBloomFilter.java | 68 ++-- .../iceberg/spark/source/TestSparkScan.java | 206 ++++++------ .../source/TestStructuredStreamingRead3.java | 165 +++++----- .../source/TestTimestampWithoutZone.java | 62 ++-- 19 files changed, 1183 insertions(+), 1018 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java index 1ec3c4726d0a..34315bc8a991 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.apache.iceberg.FileFormat.AVRO; +import static org.apache.iceberg.FileFormat.ORC; import static org.apache.iceberg.FileFormat.PARQUET; import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; @@ -34,6 +36,7 @@ import static org.apache.iceberg.spark.SparkSQLProperties.COMPRESSION_LEVEL; import static org.apache.iceberg.spark.SparkSQLProperties.COMPRESSION_STRATEGY; import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER; +import static org.assertj.core.api.Assertions.assertThat; import java.util.List; import java.util.Map; @@ -50,6 +53,9 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.actions.SizeBasedFileRewriter; @@ -58,8 +64,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalogConfig; -import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.iceberg.spark.actions.SparkActions; import org.apache.orc.OrcFile; @@ -69,66 +75,90 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestCompressionSettings extends SparkCatalogTestBase { +import org.junit.jupiter.api.AfterAll; +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; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCompressionSettings extends CatalogTestBase { private static final Configuration CONF = new Configuration(); private static final String tableName = "testWriteData"; private static SparkSession spark = null; - private final FileFormat format; - private final ImmutableMap properties; + @Parameter(index = 3) + private FileFormat format; + + @Parameter(index = 4) + private Map properties; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private java.nio.file.Path temp; - @Parameterized.Parameters(name = "format = {0}, properties = {1}") + @Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}, format = {3}, properties = {4}") public static Object[][] parameters() { return new Object[][] { - {"parquet", ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_LEVEL, "1")}, - {"parquet", ImmutableMap.of(COMPRESSION_CODEC, "gzip")}, - {"orc", ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_STRATEGY, "speed")}, - {"orc", ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_STRATEGY, "compression")}, - {"avro", ImmutableMap.of(COMPRESSION_CODEC, "snappy", COMPRESSION_LEVEL, "3")} + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + PARQUET, + ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_LEVEL, "1") + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + PARQUET, + ImmutableMap.of(COMPRESSION_CODEC, "gzip") + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + ORC, + ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_STRATEGY, "speed") + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + ORC, + ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_STRATEGY, "compression") + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + AVRO, + ImmutableMap.of(COMPRESSION_CODEC, "snappy", COMPRESSION_LEVEL, "3") + } }; } - @BeforeClass + @BeforeAll public static void startSpark() { TestCompressionSettings.spark = SparkSession.builder().master("local[2]").getOrCreate(); } - @Parameterized.AfterParam - public static void clearSourceCache() { + @AfterEach + public void afterEach() { spark.sql(String.format("DROP TABLE IF EXISTS %s", tableName)); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestCompressionSettings.spark; TestCompressionSettings.spark = null; currentSpark.stop(); } - public TestCompressionSettings(String format, ImmutableMap properties) { - super( - SparkCatalogConfig.SPARK.catalogName(), - SparkCatalogConfig.SPARK.implementation(), - SparkCatalogConfig.SPARK.properties()); - this.format = FileFormat.fromString(format); - this.properties = properties; - } - - @Test + @TestTemplate public void testWriteDataWithDifferentSetting() throws Exception { sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); Map tableProperties = Maps.newHashMap(); @@ -170,7 +200,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { try (ManifestReader reader = ManifestFiles.read(manifestFiles.get(0), table.io())) { DataFile file = reader.iterator().next(); InputFile inputFile = table.io().newInputFile(file.path().toString()); - Assertions.assertThat(getCompressionType(inputFile)) + assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -184,7 +214,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); InputFile inputFile = table.io().newInputFile(file.path().toString()); - Assertions.assertThat(getCompressionType(inputFile)) + assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -198,7 +228,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); InputFile inputFile = table.io().newInputFile(file.path().toString()); - Assertions.assertThat(getCompressionType(inputFile)) + assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java index 0efec160e8f0..348173596e46 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java @@ -21,9 +21,11 @@ import static org.apache.iceberg.Files.localOutput; import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.time.OffsetDateTime; import java.util.Arrays; import java.util.List; @@ -33,6 +35,9 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.Schema; @@ -68,18 +73,14 @@ import org.apache.spark.sql.sources.Not; import org.apache.spark.sql.sources.StringStartsWith; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestFilteredScan { private static final Configuration CONF = new Configuration(); private static final HadoopTables TABLES = new HadoopTables(CONF); @@ -107,25 +108,30 @@ public class TestFilteredScan { private static SparkSession spark = null; - @BeforeClass + @BeforeAll public static void startSpark() { TestFilteredScan.spark = SparkSession.builder().master("local[2]").getOrCreate(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestFilteredScan.spark; TestFilteredScan.spark = null; currentSpark.stop(); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - private final String format; - private final boolean vectorized; - private final PlanningMode planningMode; + @Parameter(index = 0) + private String format; - @Parameterized.Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") + @Parameter(index = 1) + private boolean vectorized; + + @Parameter(index = 2) + private PlanningMode planningMode; + + @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") public static Object[][] parameters() { return new Object[][] { {"parquet", false, LOCAL}, @@ -136,22 +142,16 @@ public static Object[][] parameters() { }; } - public TestFilteredScan(String format, boolean vectorized, PlanningMode planningMode) { - this.format = format; - this.vectorized = vectorized; - this.planningMode = planningMode; - } - private File parent = null; private File unpartitioned = null; private List records = null; - @Before + @BeforeEach public void writeUnpartitionedTable() throws IOException { - this.parent = temp.newFolder("TestFilteredScan"); + this.parent = temp.resolve("TestFilteredScan").toFile(); this.unpartitioned = new File(parent, "unpartitioned"); File dataFolder = new File(unpartitioned, "data"); - Assert.assertTrue("Mkdir should succeed", dataFolder.mkdirs()); + assertThat(dataFolder.mkdirs()).as("Mkdir should succeed").isTrue(); Table table = TABLES.create( @@ -186,7 +186,7 @@ public void writeUnpartitionedTable() throws IOException { table.newAppend().appendFile(file).commit(); } - @Test + @TestTemplate public void testUnpartitionedIDFilters() { CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); @@ -198,7 +198,7 @@ public void testUnpartitionedIDFilters() { Batch scan = builder.build().toBatch(); InputPartition[] partitions = scan.planInputPartitions(); - Assert.assertEquals("Should only create one task for a small file", 1, partitions.length); + assertThat(partitions).as("Should only create one task for a small file").hasSize(1); // validate row filtering assertEqualsSafe( @@ -206,7 +206,7 @@ public void testUnpartitionedIDFilters() { } } - @Test + @TestTemplate public void testUnpartitionedCaseInsensitiveIDFilters() { CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); @@ -228,7 +228,7 @@ public void testUnpartitionedCaseInsensitiveIDFilters() { Batch scan = builder.build().toBatch(); InputPartition[] tasks = scan.planInputPartitions(); - Assert.assertEquals("Should only create one task for a small file", 1, tasks.length); + assertThat(tasks).as("Should only create one task for a small file").hasSize(1); // validate row filtering assertEqualsSafe( @@ -242,7 +242,7 @@ public void testUnpartitionedCaseInsensitiveIDFilters() { } } - @Test + @TestTemplate public void testUnpartitionedTimestampFilter() { CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); @@ -254,7 +254,7 @@ public void testUnpartitionedTimestampFilter() { Batch scan = builder.build().toBatch(); InputPartition[] tasks = scan.planInputPartitions(); - Assert.assertEquals("Should only create one task for a small file", 1, tasks.length); + assertThat(tasks).as("Should only create one task for a small file").hasSize(1); assertEqualsSafe( SCHEMA.asStruct(), @@ -265,7 +265,7 @@ public void testUnpartitionedTimestampFilter() { "ts < cast('2017-12-22 00:00:00+00:00' as timestamp)")); } - @Test + @TestTemplate public void testBucketPartitionedIDFilters() { Table table = buildPartitionedTable("bucketed_by_id", BUCKET_BY_ID); CaseInsensitiveStringMap options = @@ -273,8 +273,9 @@ public void testBucketPartitionedIDFilters() { Batch unfiltered = new SparkScanBuilder(spark, TABLES.load(options.get("path")), options).build().toBatch(); - Assert.assertEquals( - "Unfiltered table should created 4 read tasks", 4, unfiltered.planInputPartitions().length); + assertThat(unfiltered.planInputPartitions()) + .as("Unfiltered table should created 4 read tasks") + .hasSize(4); for (int i = 0; i < 10; i += 1) { SparkScanBuilder builder = @@ -286,7 +287,7 @@ public void testBucketPartitionedIDFilters() { InputPartition[] tasks = scan.planInputPartitions(); // validate predicate push-down - Assert.assertEquals("Should create one task for a single bucket", 1, tasks.length); + assertThat(tasks).as("Should only create one task for a single bucket").hasSize(1); // validate row filtering assertEqualsSafe( @@ -295,7 +296,7 @@ public void testBucketPartitionedIDFilters() { } @SuppressWarnings("checkstyle:AvoidNestedBlocks") - @Test + @TestTemplate public void testDayPartitionedTimestampFilters() { Table table = buildPartitionedTable("partitioned_by_day", PARTITION_BY_DAY); CaseInsensitiveStringMap options = @@ -303,8 +304,9 @@ public void testDayPartitionedTimestampFilters() { Batch unfiltered = new SparkScanBuilder(spark, TABLES.load(options.get("path")), options).build().toBatch(); - Assert.assertEquals( - "Unfiltered table should created 2 read tasks", 2, unfiltered.planInputPartitions().length); + assertThat(unfiltered.planInputPartitions()) + .as("Unfiltered table should created 2 read tasks") + .hasSize(2); { SparkScanBuilder builder = @@ -314,7 +316,7 @@ public void testDayPartitionedTimestampFilters() { Batch scan = builder.build().toBatch(); InputPartition[] tasks = scan.planInputPartitions(); - Assert.assertEquals("Should create one task for 2017-12-21", 1, tasks.length); + assertThat(tasks).as("Should create one task for 2017-12-21").hasSize(1); assertEqualsSafe( SCHEMA.asStruct(), @@ -335,7 +337,7 @@ public void testDayPartitionedTimestampFilters() { Batch scan = builder.build().toBatch(); InputPartition[] tasks = scan.planInputPartitions(); - Assert.assertEquals("Should create one task for 2017-12-22", 1, tasks.length); + assertThat(tasks).as("Should create one task for 2017-12-22").hasSize(1); assertEqualsSafe( SCHEMA.asStruct(), @@ -349,7 +351,7 @@ public void testDayPartitionedTimestampFilters() { } @SuppressWarnings("checkstyle:AvoidNestedBlocks") - @Test + @TestTemplate public void testHourPartitionedTimestampFilters() { Table table = buildPartitionedTable("partitioned_by_hour", PARTITION_BY_HOUR); @@ -358,8 +360,9 @@ public void testHourPartitionedTimestampFilters() { Batch unfiltered = new SparkScanBuilder(spark, TABLES.load(options.get("path")), options).build().toBatch(); - Assert.assertEquals( - "Unfiltered table should created 9 read tasks", 9, unfiltered.planInputPartitions().length); + assertThat(unfiltered.planInputPartitions()) + .as("Unfiltered table should created 9 read tasks") + .hasSize(9); { SparkScanBuilder builder = @@ -369,7 +372,7 @@ public void testHourPartitionedTimestampFilters() { Batch scan = builder.build().toBatch(); InputPartition[] tasks = scan.planInputPartitions(); - Assert.assertEquals("Should create 4 tasks for 2017-12-21: 15, 17, 21, 22", 4, tasks.length); + assertThat(tasks).as("Should create 4 tasks for 2017-12-21: 15, 17, 21, 22").hasSize(4); assertEqualsSafe( SCHEMA.asStruct(), @@ -390,7 +393,7 @@ public void testHourPartitionedTimestampFilters() { Batch scan = builder.build().toBatch(); InputPartition[] tasks = scan.planInputPartitions(); - Assert.assertEquals("Should create 2 tasks for 2017-12-22: 6, 7", 2, tasks.length); + assertThat(tasks).as("Should create 2 tasks for 2017-12-22: 6, 7").hasSize(2); assertEqualsSafe( SCHEMA.asStruct(), @@ -404,7 +407,7 @@ public void testHourPartitionedTimestampFilters() { } @SuppressWarnings("checkstyle:AvoidNestedBlocks") - @Test + @TestTemplate public void testFilterByNonProjectedColumn() { { Schema actualProjection = SCHEMA.select("id", "data"); @@ -445,7 +448,7 @@ public void testFilterByNonProjectedColumn() { } } - @Test + @TestTemplate public void testPartitionedByDataStartsWithFilter() { Table table = buildPartitionedTable("partitioned_by_data", PARTITION_BY_DATA); CaseInsensitiveStringMap options = @@ -457,10 +460,10 @@ public void testPartitionedByDataStartsWithFilter() { pushFilters(builder, new StringStartsWith("data", "junc")); Batch scan = builder.build().toBatch(); - Assert.assertEquals(1, scan.planInputPartitions().length); + assertThat(scan.planInputPartitions()).hasSize(1); } - @Test + @TestTemplate public void testPartitionedByDataNotStartsWithFilter() { Table table = buildPartitionedTable("partitioned_by_data", PARTITION_BY_DATA); CaseInsensitiveStringMap options = @@ -472,10 +475,10 @@ public void testPartitionedByDataNotStartsWithFilter() { pushFilters(builder, new Not(new StringStartsWith("data", "junc"))); Batch scan = builder.build().toBatch(); - Assert.assertEquals(9, scan.planInputPartitions().length); + assertThat(scan.planInputPartitions()).hasSize(9); } - @Test + @TestTemplate public void testPartitionedByIdStartsWith() { Table table = buildPartitionedTable("partitioned_by_id", PARTITION_BY_ID); @@ -488,10 +491,10 @@ public void testPartitionedByIdStartsWith() { pushFilters(builder, new StringStartsWith("data", "junc")); Batch scan = builder.build().toBatch(); - Assert.assertEquals(1, scan.planInputPartitions().length); + assertThat(scan.planInputPartitions()).hasSize(1); } - @Test + @TestTemplate public void testPartitionedByIdNotStartsWith() { Table table = buildPartitionedTable("partitioned_by_id", PARTITION_BY_ID); @@ -504,10 +507,10 @@ public void testPartitionedByIdNotStartsWith() { pushFilters(builder, new Not(new StringStartsWith("data", "junc"))); Batch scan = builder.build().toBatch(); - Assert.assertEquals(9, scan.planInputPartitions().length); + assertThat(scan.planInputPartitions()).hasSize(9); } - @Test + @TestTemplate public void testUnpartitionedStartsWith() { Dataset df = spark @@ -519,11 +522,11 @@ public void testUnpartitionedStartsWith() { List matchedData = df.select("data").where("data LIKE 'jun%'").as(Encoders.STRING()).collectAsList(); - Assert.assertEquals(1, matchedData.size()); - Assert.assertEquals("junction", matchedData.get(0)); + assertThat(matchedData).hasSize(1); + assertThat(matchedData.get(0)).isEqualTo("junction"); } - @Test + @TestTemplate public void testUnpartitionedNotStartsWith() { Dataset df = spark @@ -541,8 +544,8 @@ public void testUnpartitionedNotStartsWith() { .filter(d -> !d.startsWith("jun")) .collect(Collectors.toList()); - Assert.assertEquals(9, matchedData.size()); - Assert.assertEquals(Sets.newHashSet(expected), Sets.newHashSet(matchedData)); + assertThat(matchedData).hasSize(9); + assertThat(Sets.newHashSet(matchedData)).isEqualTo(Sets.newHashSet(expected)); } private static Record projectFlat(Schema projection, Record record) { @@ -562,7 +565,7 @@ public static void assertEqualsUnsafe( for (int i = 0; i < numRecords; i += 1) { GenericsHelpers.assertEqualsUnsafe(struct, expected.get(i), actual.get(i)); } - Assert.assertEquals("Number of results should match expected", expected.size(), actual.size()); + assertThat(actual).as("Number of results should match expected").hasSameSizeAs(expected); } public static void assertEqualsSafe( @@ -572,7 +575,7 @@ public static void assertEqualsSafe( for (int i = 0; i < numRecords; i += 1) { GenericsHelpers.assertEqualsSafe(struct, expected.get(i), actual.get(i)); } - Assert.assertEquals("Number of results should match expected", expected.size(), actual.size()); + assertThat(actual).as("Number of results should match expected").hasSameSizeAs(expected); } private List expected(int... ordinals) { @@ -584,7 +587,7 @@ private List expected(int... ordinals) { } private void pushFilters(ScanBuilder scan, Filter... filters) { - Assertions.assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); + assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); SupportsPushDownV2Filters filterable = (SupportsPushDownV2Filters) scan; filterable.pushPredicates(Arrays.stream(filters).map(Filter::toV2).toArray(Predicate[]::new)); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java index 45a523917f05..0febfd687a00 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java @@ -20,14 +20,19 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Arrays; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PlanningMode; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -37,48 +42,75 @@ import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkTableUtil; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.TableIdentifier; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIdentityPartitionData extends SparkTestBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIdentityPartitionData extends TestBase { private static final Configuration CONF = new Configuration(); private static final HadoopTables TABLES = new HadoopTables(CONF); - @Parameterized.Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") + @Parameters(name = "format = {0}, vectorized = {1}, properties = {2}") public static Object[][] parameters() { return new Object[][] { - {"parquet", false, LOCAL}, - {"parquet", true, DISTRIBUTED}, - {"avro", false, LOCAL}, - {"orc", false, DISTRIBUTED}, - {"orc", true, LOCAL}, + { + "parquet", + false, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "parquet", + TableProperties.DATA_PLANNING_MODE, LOCAL.modeName(), + TableProperties.DELETE_PLANNING_MODE, LOCAL.modeName()) + }, + { + "parquet", + true, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "parquet", + TableProperties.DATA_PLANNING_MODE, DISTRIBUTED.modeName(), + TableProperties.DELETE_PLANNING_MODE, DISTRIBUTED.modeName()) + }, + { + "avro", + false, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "avro", + TableProperties.DATA_PLANNING_MODE, LOCAL.modeName(), + TableProperties.DELETE_PLANNING_MODE, LOCAL.modeName()) + }, + { + "orc", + false, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "orc", + TableProperties.DATA_PLANNING_MODE, DISTRIBUTED.modeName(), + TableProperties.DELETE_PLANNING_MODE, DISTRIBUTED.modeName()) + }, + { + "orc", + true, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "orc", + TableProperties.DATA_PLANNING_MODE, LOCAL.modeName(), + TableProperties.DELETE_PLANNING_MODE, LOCAL.modeName()) + }, }; } - private final String format; - private final boolean vectorized; - private final Map properties; + @Parameter(index = 0) + private String format; - public TestIdentityPartitionData(String format, boolean vectorized, PlanningMode planningMode) { - this.format = format; - this.vectorized = vectorized; - this.properties = - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, format, - TableProperties.DATA_PLANNING_MODE, planningMode.modeName(), - TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()); - } + @Parameter(index = 1) + private boolean vectorized; + + @Parameter(index = 2) + private Map properties; private static final Schema LOG_SCHEMA = new Schema( @@ -100,7 +132,7 @@ public TestIdentityPartitionData(String format, boolean vectorized, PlanningMode LogMessage.warn("2020-02-04", "warn event 1"), LogMessage.debug("2020-02-04", "debug event 5")); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private PartitionSpec spec = PartitionSpec.builderFor(LOG_SCHEMA).identity("date").identity("level").build(); @@ -113,10 +145,10 @@ public TestIdentityPartitionData(String format, boolean vectorized, PlanningMode * also fail. */ private void setupParquet() throws Exception { - File location = temp.newFolder("logs"); - File hiveLocation = temp.newFolder("hive"); + File location = Files.createTempDirectory(temp, "logs").toFile(); + File hiveLocation = Files.createTempDirectory(temp, "hive").toFile(); String hiveTable = "hivetable"; - Assert.assertTrue("Temp folder should exist", location.exists()); + assertThat(location).as("Temp folder should exist").exists(); this.logs = spark.createDataFrame(LOGS, LogMessage.class).select("id", "date", "level", "message"); @@ -139,13 +171,13 @@ private void setupParquet() throws Exception { spark, new TableIdentifier(hiveTable), table, location.toString()); } - @Before + @BeforeEach public void setupTable() throws Exception { if (format.equals("parquet")) { setupParquet(); } else { - File location = temp.newFolder("logs"); - Assert.assertTrue("Temp folder should exist", location.exists()); + File location = Files.createTempDirectory(temp, "logs").toFile(); + assertThat(location).as("Temp folder should exist").exists(); this.table = TABLES.create(LOG_SCHEMA, spec, properties, location.toString()); this.logs = @@ -159,7 +191,7 @@ public void setupTable() throws Exception { } } - @Test + @TestTemplate public void testFullProjection() { List expected = logs.orderBy("id").collectAsList(); List actual = @@ -171,10 +203,10 @@ public void testFullProjection() { .orderBy("id") .select("id", "date", "level", "message") .collectAsList(); - Assert.assertEquals("Rows should match", expected, actual); + assertThat(actual).as("Rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testProjections() { String[][] cases = new String[][] { @@ -210,8 +242,9 @@ public void testProjections() { .select("id", ordering) .orderBy("id") .collectAsList(); - Assert.assertEquals( - "Rows should match for ordering: " + Arrays.toString(ordering), expected, actual); + assertThat(actual) + .as("Rows should match for ordering: " + Arrays.toString(ordering)) + .isEqualTo(expected); } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java index ea65fead10fc..a417454b45dc 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java @@ -29,13 +29,18 @@ import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; import java.util.List; -import java.util.Map; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -45,25 +50,20 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalog; -import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkSessionCatalog; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class TestMetadataTablesWithPartitionEvolution extends SparkCatalogTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetadataTablesWithPartitionEvolution extends CatalogTestBase { @Parameters(name = "catalog = {0}, impl = {1}, conf = {2}, fileFormat = {3}, formatVersion = {4}") public static Object[][] parameters() { @@ -119,26 +119,18 @@ public static Object[][] parameters() { }; } - private final FileFormat fileFormat; - private final int formatVersion; - - public TestMetadataTablesWithPartitionEvolution( - String catalogName, - String implementation, - Map config, - FileFormat fileFormat, - int formatVersion) { - super(catalogName, implementation, config); - this.fileFormat = fileFormat; - this.formatVersion = formatVersion; - } + @Parameter(index = 3) + private FileFormat fileFormat; + + @Parameter(index = 4) + private int formatVersion; - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testFilesMetadataTable() throws ParseException { createTable("id bigint NOT NULL, category string, data string"); @@ -147,8 +139,9 @@ public void testFilesMetadataTable() throws ParseException { // verify the metadata tables while the current spec is still unpartitioned for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { Dataset df = loadMetadataTable(tableType); - Assert.assertTrue( - "Partition must be skipped", df.schema().getFieldIndex("partition").isEmpty()); + assertThat(df.schema().getFieldIndex("partition").isEmpty()) + .as("Partition must be skipped") + .isTrue(); } Table table = validationCatalog.loadTable(tableIdent); @@ -199,7 +192,7 @@ public void testFilesMetadataTable() throws ParseException { } } - @Test + @TestTemplate public void testFilesMetadataTableFilter() throws ParseException { createTable("id bigint NOT NULL, category string, data string"); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, MANIFEST_MERGE_ENABLED); @@ -210,8 +203,9 @@ public void testFilesMetadataTableFilter() throws ParseException { // verify the metadata tables while the current spec is still unpartitioned for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { Dataset df = loadMetadataTable(tableType); - Assert.assertTrue( - "Partition must be skipped", df.schema().getFieldIndex("partition").isEmpty()); + assertThat(df.schema().getFieldIndex("partition").isEmpty()) + .as("Partition must be skipped") + .isTrue(); } Table table = validationCatalog.loadTable(tableIdent); @@ -290,7 +284,7 @@ public void testFilesMetadataTableFilter() throws ParseException { } } - @Test + @TestTemplate public void testEntriesMetadataTable() throws ParseException { createTable("id bigint NOT NULL, category string, data string"); @@ -300,7 +294,7 @@ public void testEntriesMetadataTable() throws ParseException { for (MetadataTableType tableType : Arrays.asList(ENTRIES, ALL_ENTRIES)) { Dataset df = loadMetadataTable(tableType); StructType dataFileType = (StructType) df.schema().apply("data_file").dataType(); - Assert.assertTrue("Partition must be skipped", dataFileType.getFieldIndex("").isEmpty()); + assertThat(dataFileType.getFieldIndex("").isEmpty()).as("Partition must be skipped").isTrue(); } Table table = validationCatalog.loadTable(tableIdent); @@ -351,7 +345,7 @@ public void testEntriesMetadataTable() throws ParseException { } } - @Test + @TestTemplate public void testPartitionsTableAddRemoveFields() throws ParseException { createTable("id bigint NOT NULL, category string, data string"); sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); @@ -359,8 +353,9 @@ public void testPartitionsTableAddRemoveFields() throws ParseException { // verify the metadata tables while the current spec is still unpartitioned Dataset df = loadMetadataTable(PARTITIONS); - Assert.assertTrue( - "Partition must be skipped", df.schema().getFieldIndex("partition").isEmpty()); + assertThat(df.schema().getFieldIndex("partition").isEmpty()) + .as("Partition must be skipped") + .isTrue(); Table table = validationCatalog.loadTable(tableIdent); @@ -406,7 +401,7 @@ public void testPartitionsTableAddRemoveFields() throws ParseException { PARTITIONS); } - @Test + @TestTemplate public void testPartitionsTableRenameFields() throws ParseException { createTable("id bigint NOT NULL, category string, data string"); @@ -433,7 +428,7 @@ public void testPartitionsTableRenameFields() throws ParseException { PARTITIONS); } - @Test + @TestTemplate public void testPartitionsTableSwitchFields() throws Exception { createTable("id bigint NOT NULL, category string, data string"); @@ -495,7 +490,7 @@ public void testPartitionsTableSwitchFields() throws Exception { } } - @Test + @TestTemplate public void testPartitionTableFilterAddRemoveFields() throws ParseException { // Create un-partitioned table createTable("id bigint NOT NULL, category string, data string"); @@ -549,13 +544,13 @@ public void testPartitionTableFilterAddRemoveFields() throws ParseException { "partition.category = 'c2'"); } - @Test + @TestTemplate public void testPartitionTableFilterSwitchFields() throws Exception { // Re-added partition fields currently not re-associated: // https://github.com/apache/iceberg/issues/4292 // In V1, dropped partition fields show separately when field is re-added // In V2, re-added field currently conflicts with its deleted form - Assume.assumeTrue(formatVersion == 1); + assumeThat(formatVersion).isEqualTo(1); createTable("id bigint NOT NULL, category string, data string"); sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); @@ -595,7 +590,7 @@ public void testPartitionTableFilterSwitchFields() throws Exception { "partition.data = 'd1'"); } - @Test + @TestTemplate public void testPartitionsTableFilterRenameFields() throws ParseException { createTable("id bigint NOT NULL, category string, data string"); @@ -618,7 +613,7 @@ public void testPartitionsTableFilterRenameFields() throws ParseException { "partition.category_another_name = 'c1'"); } - @Test + @TestTemplate public void testMetadataTablesWithUnknownTransforms() { createTable("id bigint NOT NULL, category string, data string"); @@ -641,13 +636,13 @@ public void testMetadataTablesWithUnknownTransforms() { sql("REFRESH TABLE %s", tableName); for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES, ENTRIES, ALL_ENTRIES)) { - Assertions.assertThatThrownBy(() -> loadMetadataTable(tableType)) + assertThatThrownBy(() -> loadMetadataTable(tableType)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot build table partition type, unknown transforms: [zero]"); } } - @Test + @TestTemplate public void testPartitionColumnNamedPartition() { sql( "CREATE TABLE %s (id int, partition int) USING iceberg PARTITIONED BY (partition)", @@ -655,7 +650,7 @@ public void testPartitionColumnNamedPartition() { sql("INSERT INTO %s VALUES (1, 1), (2, 1), (3, 2), (2, 2)", tableName); List expected = ImmutableList.of(row(1, 1), row(2, 1), row(3, 2), row(2, 2)); assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); - Assert.assertEquals(2, sql("SELECT * FROM %s.files", tableName).size()); + assertThat(sql("SELECT * FROM %s.files", tableName)).hasSize(2); } private void assertPartitions( @@ -681,14 +676,14 @@ private void assertPartitions( case FILES: case ALL_DATA_FILES: DataType actualFilesType = df.schema().apply("partition").dataType(); - Assert.assertEquals("Partition type must match", expectedType, actualFilesType); + assertThat(actualFilesType).as("Partition type must match").isEqualTo(expectedType); break; case ENTRIES: case ALL_ENTRIES: StructType dataFileType = (StructType) df.schema().apply("data_file").dataType(); DataType actualEntriesType = dataFileType.apply("partition").dataType(); - Assert.assertEquals("Partition type must match", expectedType, actualEntriesType); + assertThat(actualEntriesType).as("Partition type must match").isEqualTo(expectedType); break; default: diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java index c00549c68f3b..2bd6dadff016 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java @@ -20,10 +20,12 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; import java.net.URI; +import java.nio.file.Files; import java.sql.Timestamp; import java.time.Instant; import java.util.Arrays; @@ -39,6 +41,9 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.Schema; @@ -63,22 +68,19 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.unsafe.types.UTF8String; -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 org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestPartitionPruning { private static final Configuration CONF = new Configuration(); private static final HadoopTables TABLES = new HadoopTables(CONF); - @Parameterized.Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") + @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") public static Object[][] parameters() { return new Object[][] { {"parquet", false, DISTRIBUTED}, @@ -89,15 +91,14 @@ public static Object[][] parameters() { }; } - private final String format; - private final boolean vectorized; - private final PlanningMode planningMode; + @Parameter(index = 0) + private String format; - public TestPartitionPruning(String format, boolean vectorized, PlanningMode planningMode) { - this.format = format; - this.vectorized = vectorized; - this.planningMode = planningMode; - } + @Parameter(index = 1) + private boolean vectorized; + + @Parameter(index = 2) + private PlanningMode planningMode; private static SparkSession spark = null; private static JavaSparkContext sparkContext = null; @@ -109,7 +110,7 @@ public TestPartitionPruning(String format, boolean vectorized, PlanningMode plan private static final Function HOUR_FUNC = Transforms.hour().bind(Types.TimestampType.withoutZone()); - @BeforeClass + @BeforeAll public static void startSpark() { TestPartitionPruning.spark = SparkSession.builder().master("local[2]").getOrCreate(); TestPartitionPruning.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); @@ -133,7 +134,7 @@ public static void startSpark() { DataTypes.IntegerType); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestPartitionPruning.spark; TestPartitionPruning.spark = null; @@ -167,7 +168,7 @@ private static Instant getInstant(String timestampWithoutZone) { return Instant.ofEpochMilli(TimeUnit.MICROSECONDS.toMillis(epochMicros)); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private java.nio.file.Path temp; private PartitionSpec spec = PartitionSpec.builderFor(LOG_SCHEMA) @@ -178,7 +179,7 @@ private static Instant getInstant(String timestampWithoutZone) { .hour("timestamp") .build(); - @Test + @TestTemplate public void testPartitionPruningIdentityString() { String filterCond = "date >= '2020-02-03' AND level = 'DEBUG'"; Predicate partCondition = @@ -191,7 +192,7 @@ public void testPartitionPruningIdentityString() { runTest(filterCond, partCondition); } - @Test + @TestTemplate public void testPartitionPruningBucketingInteger() { final int[] ids = new int[] {LOGS.get(3).getId(), LOGS.get(7).getId()}; String condForIds = @@ -208,7 +209,7 @@ public void testPartitionPruningBucketingInteger() { runTest(filterCond, partCondition); } - @Test + @TestTemplate public void testPartitionPruningTruncatedString() { String filterCond = "message like 'info event%'"; Predicate partCondition = @@ -220,7 +221,7 @@ public void testPartitionPruningTruncatedString() { runTest(filterCond, partCondition); } - @Test + @TestTemplate public void testPartitionPruningTruncatedStringComparingValueShorterThanPartitionValue() { String filterCond = "message like 'inf%'"; Predicate partCondition = @@ -232,7 +233,7 @@ public void testPartitionPruningTruncatedStringComparingValueShorterThanPartitio runTest(filterCond, partCondition); } - @Test + @TestTemplate public void testPartitionPruningHourlyPartition() { String filterCond; if (spark.version().startsWith("2")) { @@ -256,7 +257,7 @@ public void testPartitionPruningHourlyPartition() { private void runTest(String filterCond, Predicate partCondition) { File originTableLocation = createTempDir(); - Assert.assertTrue("Temp folder should exist", originTableLocation.exists()); + assertThat(originTableLocation).as("Temp folder should exist").exists(); Table table = createTable(originTableLocation); Dataset logs = createTestDataset(); @@ -267,7 +268,7 @@ private void runTest(String filterCond, Predicate partCondition) { .filter(filterCond) .orderBy("id") .collectAsList(); - Assert.assertFalse("Expected rows should be not empty", expected.isEmpty()); + assertThat(expected).as("Expected rows should not be empty").isNotEmpty(); // remove records which may be recorded during storing to table CountOpenLocalFileSystem.resetRecordsInPathPrefix(originTableLocation.getAbsolutePath()); @@ -282,16 +283,16 @@ private void runTest(String filterCond, Predicate partCondition) { .filter(filterCond) .orderBy("id") .collectAsList(); - Assert.assertFalse("Actual rows should not be empty", actual.isEmpty()); + assertThat(actual).as("Actual rows should not be empty").isNotEmpty(); - Assert.assertEquals("Rows should match", expected, actual); + assertThat(actual).as("Rows should match").isEqualTo(expected); assertAccessOnDataFiles(originTableLocation, table, partCondition); } private File createTempDir() { try { - return temp.newFolder(); + return Files.createTempDirectory(temp, "junit").toFile(); } catch (Exception e) { throw new RuntimeException(e); } @@ -366,29 +367,31 @@ private void assertAccessOnDataFiles( Set filesToNotRead = extractFilePathsNotIn(files, filesToRead); // Just to be sure, they should be mutually exclusive. - Assert.assertTrue(Sets.intersection(filesToRead, filesToNotRead).isEmpty()); + assertThat(filesToRead).doesNotContainAnyElementsOf(filesToNotRead); - Assert.assertFalse("The query should prune some data files.", filesToNotRead.isEmpty()); + assertThat(filesToNotRead).as("The query should prune some data files.").isNotEmpty(); // We don't check "all" data files bound to the condition are being read, as data files can be // pruned on // other conditions like lower/upper bound of columns. - Assert.assertFalse( - "Some of data files in partition range should be read. " - + "Read files in query: " - + readFilesInQuery - + " / data files in partition range: " - + filesToRead, - Sets.intersection(filesToRead, readFilesInQuery).isEmpty()); + assertThat(filesToRead) + .as( + "Some of data files in partition range should be read. " + + "Read files in query: " + + readFilesInQuery + + " / data files in partition range: " + + filesToRead) + .containsAnyElementsOf(readFilesInQuery); // Data files which aren't bound to the condition shouldn't be read. - Assert.assertTrue( - "Data files outside of partition range should not be read. " - + "Read files in query: " - + readFilesInQuery - + " / data files outside of partition range: " - + filesToNotRead, - Sets.intersection(filesToNotRead, readFilesInQuery).isEmpty()); + assertThat(filesToNotRead) + .as( + "Data files outside of partition range should not be read. " + + "Read files in query: " + + readFilesInQuery + + " / data files outside of partition range: " + + filesToNotRead) + .doesNotContainAnyElementsOf(readFilesInQuery); } private Set extractFilePathsMatchingConditionOnPartition( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java index 11153b3943b4..5c218f21c47e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java @@ -18,16 +18,23 @@ */ package org.apache.iceberg.spark.source; +import static org.apache.iceberg.spark.data.TestVectorizedOrcDataReader.temp; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; +import java.nio.file.Path; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.avro.generic.GenericData; import org.apache.iceberg.DataFiles; import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -52,19 +59,15 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestPartitionValues { - @Parameterized.Parameters(name = "format = {0}, vectorized = {1}") + @Parameters(name = "format = {0}, vectorized = {1}") public static Object[][] parameters() { return new Object[][] { {"parquet", false}, @@ -102,35 +105,33 @@ public static Object[][] parameters() { private static SparkSession spark = null; - @BeforeClass + @BeforeAll public static void startSpark() { TestPartitionValues.spark = SparkSession.builder().master("local[2]").getOrCreate(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestPartitionValues.spark; TestPartitionValues.spark = null; currentSpark.stop(); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - private final String format; - private final boolean vectorized; + @Parameter(index = 0) + private String format; - public TestPartitionValues(String format, boolean vectorized) { - this.format = format; - this.vectorized = vectorized; - } + @Parameter(index = 1) + private boolean vectorized; - @Test + @TestTemplate public void testNullPartitionValue() throws Exception { String desc = "null_part"; - File parent = temp.newFolder(desc); + File parent = new File(temp.toFile(), desc); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); - Assert.assertTrue("mkdirs should succeed", dataFolder.mkdirs()); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); Table table = tables.create(SIMPLE_SCHEMA, SPEC, location.toString()); @@ -161,17 +162,17 @@ public void testNullPartitionValue() throws Exception { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testReorderedColumns() throws Exception { String desc = "reorder_columns"; - File parent = temp.newFolder(desc); + File parent = new File(temp.toFile(), desc); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); - Assert.assertTrue("mkdirs should succeed", dataFolder.mkdirs()); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); Table table = tables.create(SIMPLE_SCHEMA, SPEC, location.toString()); @@ -200,17 +201,17 @@ public void testReorderedColumns() throws Exception { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testReorderedColumnsNoNullability() throws Exception { String desc = "reorder_columns_no_nullability"; - File parent = temp.newFolder(desc); + File parent = new File(temp.toFile(), desc); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); - Assert.assertTrue("mkdirs should succeed", dataFolder.mkdirs()); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); Table table = tables.create(SIMPLE_SCHEMA, SPEC, location.toString()); @@ -240,11 +241,11 @@ public void testReorderedColumnsNoNullability() throws Exception { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testPartitionValueTypes() throws Exception { String[] columnNames = new String[] { @@ -254,13 +255,13 @@ public void testPartitionValueTypes() throws Exception { HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); // create a table around the source data - String sourceLocation = temp.newFolder("source_table").toString(); + String sourceLocation = temp.resolve("source_table").toString(); Table source = tables.create(SUPPORTED_PRIMITIVES, sourceLocation); // write out an Avro data file with all of the data types for source data List expected = RandomData.generateList(source.schema(), 2, 128735L); - File avroData = temp.newFile("data.avro"); - Assert.assertTrue(avroData.delete()); + File avroData = File.createTempFile("data", ".avro", temp.toFile()); + assertThat(avroData.delete()).isTrue(); try (FileAppender appender = Avro.write(Files.localOutput(avroData)).schema(source.schema()).build()) { appender.addAll(expected); @@ -286,10 +287,10 @@ public void testPartitionValueTypes() throws Exception { for (String column : columnNames) { String desc = "partition_by_" + SUPPORTED_PRIMITIVES.findType(column).toString(); - File parent = temp.newFolder(desc); + File parent = new File(temp.toFile(), desc); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); - Assert.assertTrue("mkdirs should succeed", dataFolder.mkdirs()); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); PartitionSpec spec = PartitionSpec.builderFor(SUPPORTED_PRIMITIVES).identity(column).build(); @@ -313,7 +314,7 @@ public void testPartitionValueTypes() throws Exception { .load(location.toString()) .collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); for (int i = 0; i < expected.size(); i += 1) { TestHelpers.assertEqualsSafe( @@ -322,7 +323,7 @@ public void testPartitionValueTypes() throws Exception { } } - @Test + @TestTemplate public void testNestedPartitionValues() throws Exception { String[] columnNames = new String[] { @@ -333,13 +334,13 @@ public void testNestedPartitionValues() throws Exception { Schema nestedSchema = new Schema(optional(1, "nested", SUPPORTED_PRIMITIVES.asStruct())); // create a table around the source data - String sourceLocation = temp.newFolder("source_table").toString(); + String sourceLocation = temp.resolve("source_table").toString(); Table source = tables.create(nestedSchema, sourceLocation); // write out an Avro data file with all of the data types for source data List expected = RandomData.generateList(source.schema(), 2, 128735L); - File avroData = temp.newFile("data.avro"); - Assert.assertTrue(avroData.delete()); + File avroData = File.createTempFile("data", ".avro", temp.toFile()); + assertThat(avroData.delete()).isTrue(); try (FileAppender appender = Avro.write(Files.localOutput(avroData)).schema(source.schema()).build()) { appender.addAll(expected); @@ -365,10 +366,10 @@ public void testNestedPartitionValues() throws Exception { for (String column : columnNames) { String desc = "partition_by_" + SUPPORTED_PRIMITIVES.findType(column).toString(); - File parent = temp.newFolder(desc); + File parent = new File(temp.toFile(), desc); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); - Assert.assertTrue("mkdirs should succeed", dataFolder.mkdirs()); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); PartitionSpec spec = PartitionSpec.builderFor(nestedSchema).identity("nested." + column).build(); @@ -393,7 +394,7 @@ public void testNestedPartitionValues() throws Exception { .load(location.toString()) .collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); for (int i = 0; i < expected.size(); i += 1) { TestHelpers.assertEqualsSafe(nestedSchema.asStruct(), expected.get(i), actual.get(i)); @@ -407,7 +408,7 @@ public void testNestedPartitionValues() throws Exception { * thrown with the message like: Cannot cast org.apache.spark.unsafe.types.UTF8String to * java.lang.CharSequence */ - @Test + @TestTemplate public void testPartitionedByNestedString() throws Exception { // schema and partition spec Schema nestedSchema = @@ -421,7 +422,7 @@ public void testPartitionedByNestedString() throws Exception { // create table HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); - String baseLocation = temp.newFolder("partition_by_nested_string").toString(); + String baseLocation = temp.resolve("partition_by_nested_string").toString(); tables.create(nestedSchema, spec, baseLocation); // input data frame @@ -452,12 +453,12 @@ public void testPartitionedByNestedString() throws Exception { .load(baseLocation) .collectAsList(); - Assert.assertEquals("Number of rows should match", rows.size(), actual.size()); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(rows); } - @Test + @TestTemplate public void testReadPartitionColumn() throws Exception { - Assume.assumeTrue("Temporary skip ORC", !"orc".equals(format)); + assumeThat(format).as("Temporary skip ORC").isNotEqualTo("orc"); Schema nestedSchema = new Schema( @@ -473,7 +474,7 @@ public void testReadPartitionColumn() throws Exception { // create table HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); - String baseLocation = temp.newFolder("partition_by_nested_string").toString(); + String baseLocation = temp.resolve("partition_by_nested_string").toString(); Table table = tables.create(nestedSchema, spec, baseLocation); table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit(); @@ -499,10 +500,10 @@ public void testReadPartitionColumn() throws Exception { .as(Encoders.STRING()) .collectAsList(); - Assert.assertEquals("Number of rows should match", 10, actual.size()); + assertThat(actual).as("Number of rows should match").hasSize(10); List inputRecords = IntStream.range(0, 10).mapToObj(i -> "name_" + i).collect(Collectors.toList()); - Assert.assertEquals("Read object should be matched", inputRecords, actual); + assertThat(actual).as("Read object should be matched").isEqualTo(inputRecords); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java index 2ec4f2f4f907..a6573171aa6a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; import java.io.IOException; import java.util.Comparator; import java.util.List; @@ -33,6 +37,9 @@ import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Partitioning; @@ -54,12 +61,11 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator; import org.apache.iceberg.spark.ScanTaskSetManager; import org.apache.iceberg.spark.SparkCatalogConfig; -import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkStructLike; import org.apache.iceberg.spark.SparkWriteOptions; @@ -73,15 +79,11 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.functions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestPositionDeletesTable extends SparkCatalogTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestPositionDeletesTable extends CatalogTestBase { public static final Schema SCHEMA = new Schema( @@ -95,11 +97,10 @@ public class TestPositionDeletesTable extends SparkCatalogTestBase { private static final List NON_PATH_COLS = ImmutableList.of("file_path", "pos", "row", "partition", "spec_id"); - private final FileFormat format; + @Parameter(index = 3) + private FileFormat format; - @Parameterized.Parameters( - name = - "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}, fileFormat = {4}") + @Parameters(name = "catalogName = {1}, implementation = {2}, config = {3}, fileFormat = {4}") public static Object[][] parameters() { return new Object[][] { { @@ -123,15 +124,7 @@ public static Object[][] parameters() { }; } - public TestPositionDeletesTable( - String catalogName, String implementation, Map config, FileFormat format) { - super(catalogName, implementation, config); - this.format = format; - } - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - @Test + @TestTemplate public void testNullRows() throws IOException { String tableName = "null_rows"; Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); @@ -144,7 +137,10 @@ public void testNullRows() throws IOException { deletes.add(Pair.of(dFile.path(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( - tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes); tab.newRowDelta().addDeletes(posDeletes.first()).commit(); StructLikeSet actual = actual(tableName, tab); @@ -154,11 +150,11 @@ public void testNullRows() throws IOException { StructLikeSet expected = expected(tab, expectedDeletes, null, posDeletes.first().path().toString()); - Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); dropTable(tableName); } - @Test + @TestTemplate public void testPartitionedTable() throws IOException { // Create table with two partitions String tableName = "partitioned_table"; @@ -183,11 +179,11 @@ public void testPartitionedTable() throws IOException { StructLikeSet expected = expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); - Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); dropTable(tableName); } - @Test + @TestTemplate public void testSelect() throws IOException { // Create table with two partitions String tableName = "select"; @@ -244,11 +240,14 @@ public void testSelect() throws IOException { }; actual.sort(comp); expected.sort(comp); - assertEquals("Position Delete table should contain expected rows", expected, actual); + assertThat(actual) + .as("Position Delete table should contain expected rows") + .usingRecursiveComparison() + .isEqualTo(expected); dropTable(tableName); } - @Test + @TestTemplate public void testSplitTasks() throws IOException { String tableName = "big_table"; Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); @@ -263,7 +262,7 @@ public void testSplitTasks() throws IOException { DataFile dFile = FileHelpers.writeDataFile( tab, - Files.localOutput(temp.newFile()), + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), org.apache.iceberg.TestHelpers.Row.of(), dataRecords); tab.newAppend().appendFile(dFile).commit(); @@ -274,31 +273,33 @@ public void testSplitTasks() throws IOException { } DeleteFile posDeletes = FileHelpers.writePosDeleteFile( - tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes); tab.newRowDelta().addDeletes(posDeletes).commit(); Table deleteTable = MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); if (format.equals(FileFormat.AVRO)) { - Assert.assertTrue( - "Position delete scan should produce more than one split", - Iterables.size(deleteTable.newBatchScan().planTasks()) > 1); + assertThat(deleteTable.newBatchScan().planTasks()) + .as("Position delete scan should produce more than one split") + .hasSizeGreaterThan(1); } else { - Assert.assertEquals( - "Position delete scan should produce one split", - 1, - Iterables.size(deleteTable.newBatchScan().planTasks())); + assertThat(deleteTable.newBatchScan().planTasks()) + .as("Position delete scan should produce one split") + .hasSize(1); } StructLikeSet actual = actual(tableName, tab); StructLikeSet expected = expected(tab, deletes, null, posDeletes.path().toString()); - Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); dropTable(tableName); } - @Test + @TestTemplate public void testPartitionFilter() throws IOException { // Create table with two partitions String tableName = "partition_filter"; @@ -331,16 +332,20 @@ public void testPartitionFilter() throws IOException { // Select deletes from all partitions StructLikeSet actual = actual(tableName, tab); - Assert.assertEquals("Position Delete table should contain expected rows", allExpected, actual); + assertThat(actual) + .as("Position Delete table should contain expected rows") + .isEqualTo(allExpected); // Select deletes from one partition StructLikeSet actual2 = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actual2); + assertThat(actual2) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); dropTable(tableName); } - @Test + @TestTemplate public void testPartitionTransformFilter() throws IOException { // Create table with two partitions String tableName = "partition_filter"; @@ -374,16 +379,20 @@ public void testPartitionTransformFilter() throws IOException { // Select deletes from all partitions StructLikeSet actual = actual(tableName, tab); - Assert.assertEquals("Position Delete table should contain expected rows", allExpected, actual); + assertThat(actual) + .as("Position Delete table should contain expected rows") + .isEqualTo(allExpected); // Select deletes from one partition StructLikeSet actual2 = actual(tableName, tab, "partition.data_trunc = 'a' AND pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actual2); + assertThat(actual2) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); dropTable(tableName); } - @Test + @TestTemplate public void testPartitionEvolutionReplace() throws Exception { // Create table with spec (data) String tableName = "partition_evolution"; @@ -417,7 +426,9 @@ public void testPartitionEvolutionReplace() throws Exception { StructLikeSet expectedA = expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); // Query partition of new spec Record partition10 = partitionRecordTemplate.copy("id", 10); @@ -430,11 +441,13 @@ public void testPartitionEvolutionReplace() throws Exception { deletes10.second().path().toString()); StructLikeSet actual10 = actual(tableName, tab, "partition.id = 10 AND pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expected10, actual10); + assertThat(actual10) + .as("Position Delete table should contain expected rows") + .isEqualTo(expected10); dropTable(tableName); } - @Test + @TestTemplate public void testPartitionEvolutionAdd() throws Exception { // Create unpartitioned table String tableName = "partition_evolution_add"; @@ -467,7 +480,9 @@ public void testPartitionEvolutionAdd() throws Exception { StructLikeSet expectedA = expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().path().toString()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); // Select deletes from 'unpartitioned' data Record unpartitionedRecord = partitionRecordTemplate.copy("data", null); @@ -481,14 +496,13 @@ public void testPartitionEvolutionAdd() throws Exception { StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); - Assert.assertEquals( - "Position Delete table should contain expected rows", - expectedUnpartitioned, - actualUnpartitioned); + assertThat(actualUnpartitioned) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedUnpartitioned); dropTable(tableName); } - @Test + @TestTemplate public void testPartitionEvolutionRemove() throws Exception { // Create table with spec (data) String tableName = "partition_evolution_remove"; @@ -522,7 +536,9 @@ public void testPartitionEvolutionRemove() throws Exception { StructLikeSet expectedA = expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().path().toString()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); // Select deletes from 'unpartitioned' spec Record unpartitionedRecord = partitionRecordTemplate.copy("data", null); @@ -536,14 +552,13 @@ public void testPartitionEvolutionRemove() throws Exception { StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); - Assert.assertEquals( - "Position Delete table should contain expected rows", - expectedUnpartitioned, - actualUnpartitioned); + assertThat(actualUnpartitioned) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedUnpartitioned); dropTable(tableName); } - @Test + @TestTemplate public void testSpecIdFilter() throws Exception { // Create table with spec (data) String tableName = "spec_id_filter"; @@ -580,10 +595,9 @@ public void testSpecIdFilter() throws Exception { deletesUnpartitioned.second().path().toString()); StructLikeSet actualUnpartitioned = actual(tableName, tab, String.format("spec_id = %d", unpartitionedSpec)); - Assert.assertEquals( - "Position Delete table should contain expected rows", - expectedUnpartitioned, - actualUnpartitioned); + assertThat(actualUnpartitioned) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedUnpartitioned); // Select deletes from 'data' partition spec StructLike partitionA = partitionRecordTemplate.copy("data", "a"); @@ -594,11 +608,11 @@ public void testSpecIdFilter() throws Exception { expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().path().toString())); StructLikeSet actual = actual(tableName, tab, String.format("spec_id = %d", dataSpec)); - Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); dropTable(tableName); } - @Test + @TestTemplate public void testSchemaEvolutionAdd() throws Exception { // Create table with original schema String tableName = "schema_evolution_add"; @@ -647,7 +661,9 @@ public void testSchemaEvolutionAdd() throws Exception { StructLikeSet expectedA = expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); @@ -655,11 +671,13 @@ public void testSchemaEvolutionAdd() throws Exception { expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); + assertThat(actualC) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedC); dropTable(tableName); } - @Test + @TestTemplate public void testSchemaEvolutionRemove() throws Exception { // Create table with original schema String tableName = "schema_evolution_remove"; @@ -709,7 +727,9 @@ public void testSchemaEvolutionRemove() throws Exception { StructLikeSet expectedA = expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); @@ -717,11 +737,13 @@ public void testSchemaEvolutionRemove() throws Exception { expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); - Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); + assertThat(actualC) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedC); dropTable(tableName); } - @Test + @TestTemplate public void testWrite() throws IOException, NoSuchTableException { String tableName = "test_write"; PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); @@ -753,7 +775,7 @@ public void testWrite() throws IOException, NoSuchTableException { .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) .load(posDeletesTableName); - Assert.assertEquals(1, scanDF.javaRDD().getNumPartitions()); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); scanDF .writeTo(posDeletesTableName) .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) @@ -779,11 +801,13 @@ public void testWrite() throws IOException, NoSuchTableException { // Compare values without 'delete_file_path' as these have been rewritten StructLikeSet actual = actual(tableName, tab, null, NON_PATH_COLS); - Assert.assertEquals("Position Delete table should contain expected rows", allExpected, actual); + assertThat(actual) + .as("Position Delete table should contain expected rows") + .isEqualTo(allExpected); dropTable(tableName); } - @Test + @TestTemplate public void testWriteUnpartitionedNullRows() throws Exception { String tableName = "write_null_rows"; Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); @@ -796,7 +820,10 @@ public void testWriteUnpartitionedNullRows() throws Exception { deletes.add(Pair.of(dFile.path(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( - tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes); tab.newRowDelta().addDeletes(posDeletes.first()).commit(); Table posDeletesTable = @@ -813,7 +840,7 @@ public void testWriteUnpartitionedNullRows() throws Exception { .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) .load(posDeletesTableName); - Assert.assertEquals(1, scanDF.javaRDD().getNumPartitions()); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); scanDF .writeTo(posDeletesTableName) .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) @@ -830,11 +857,11 @@ public void testWriteUnpartitionedNullRows() throws Exception { Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); StructLikeSet expected = expected(tab, expectedDeletes, null, null); - Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); dropTable(tableName); } - @Test + @TestTemplate public void testWriteMixedRows() throws Exception { String tableName = "write_mixed_rows"; PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); @@ -850,7 +877,10 @@ public void testWriteMixedRows() throws Exception { deletes.add(Pair.of(dataFileA.path(), 1L)); Pair deletesWithoutRow = FileHelpers.writeDeleteFile( - tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of("a"), deletes); + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of("a"), + deletes); Pair>, DeleteFile> deletesWithRow = deleteFile(tab, dataFileB, "b"); @@ -874,7 +904,7 @@ public void testWriteMixedRows() throws Exception { .format("iceberg") .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) .load(posDeletesTableName); - Assert.assertEquals(1, scanDF.javaRDD().getNumPartitions()); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); scanDF .writeTo(posDeletesTableName) .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) @@ -910,11 +940,13 @@ public void testWriteMixedRows() throws Exception { null)); allExpected.addAll(expected(tab, deletesWithRow.first(), partitionB, null)); - Assert.assertEquals("Position Delete table should contain expected rows", allExpected, actual); + assertThat(actual) + .as("Position Delete table should contain expected rows") + .isEqualTo(allExpected); dropTable(tableName); } - @Test + @TestTemplate public void testWritePartitionEvolutionAdd() throws Exception { // Create unpartitioned table String tableName = "write_partition_evolution_add"; @@ -958,7 +990,7 @@ public void testWritePartitionEvolutionAdd() throws Exception { .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) .load(posDeletesTableName); - Assert.assertEquals(1, scanDF.javaRDD().getNumPartitions()); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); scanDF .writeTo(posDeletesTableName) .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) @@ -975,10 +1007,9 @@ public void testWritePartitionEvolutionAdd() throws Exception { expected(tab, deletesUnpartitioned.first(), unpartitionedRecord, specId0, null); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL", NON_PATH_COLS); - Assert.assertEquals( - "Position Delete table should contain expected rows", - expectedUnpartitioned, - actualUnpartitioned); + assertThat(actualUnpartitioned) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedUnpartitioned); // Read/write back new partition spec (data) for (String partValue : ImmutableList.of("a", "b")) { @@ -993,7 +1024,7 @@ public void testWritePartitionEvolutionAdd() throws Exception { .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) .load(posDeletesTableName); - Assert.assertEquals(1, scanDF.javaRDD().getNumPartitions()); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); scanDF .writeTo(posDeletesTableName) .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) @@ -1016,13 +1047,13 @@ public void testWritePartitionEvolutionAdd() throws Exception { expectedAll.addAll(expected(tab, deletesB.first(), partitionB, specId1, null)); StructLikeSet actualAll = actual(tableName, tab, "partition.data = 'a' OR partition.data = 'b'", NON_PATH_COLS); - Assert.assertEquals( - "Position Delete table should contain expected rows", expectedAll, actualAll); - + assertThat(actualAll) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedAll); dropTable(tableName); } - @Test + @TestTemplate public void testWritePartitionEvolutionDisallowed() throws Exception { // Create unpartitioned table String tableName = "write_partition_evolution_write"; @@ -1051,24 +1082,31 @@ public void testWritePartitionEvolutionDisallowed() throws Exception { .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) .load(posDeletesTableName); - Assert.assertEquals(1, scanDF.javaRDD().getNumPartitions()); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); // Add partition field to render the original un-partitioned dataset un-commitable tab.updateSpec().addField("data").commit(); } - Assert.assertThrows( - AnalysisException.class, - () -> - scanDF - .writeTo(posDeletesTableName) - .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) - .append()); + assertThatThrownBy( + () -> + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append()) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA] Cannot write incompatible data for the table `" + + catalogName + + "`.`default`.`" + + tableName + + "`.`position_deletes`" + + ": Cannot find data for the output column `partition`."); dropTable(tableName); } - @Test + @TestTemplate public void testWriteSchemaEvolutionAdd() throws Exception { // Create table with original schema String tableName = "write_schema_evolution_add"; @@ -1116,7 +1154,7 @@ public void testWriteSchemaEvolutionAdd() throws Exception { .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) .load(posDeletesTableName); - Assert.assertEquals(1, scanDF.javaRDD().getNumPartitions()); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); scanDF .writeTo(posDeletesTableName) .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) @@ -1142,7 +1180,9 @@ public void testWriteSchemaEvolutionAdd() throws Exception { }); StructLikeSet expectedA = expected(tab, expectedDeletesA, partitionA, null); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a'", NON_PATH_COLS); - Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); // rewrite files of new schema try (CloseableIterable tasks = tasks(posDeletesTable, "data", "c")) { @@ -1157,7 +1197,7 @@ public void testWriteSchemaEvolutionAdd() throws Exception { .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) .load(posDeletesTableName); - Assert.assertEquals(1, scanDF.javaRDD().getNumPartitions()); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); scanDF .writeTo(posDeletesTableName) .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) @@ -1171,11 +1211,13 @@ public void testWriteSchemaEvolutionAdd() throws Exception { StructLikeSet expectedC = expected(tab, deletesC.first(), partitionC, null); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c'", NON_PATH_COLS); - Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); + assertThat(actualC) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedC); dropTable(tableName); } - @Test + @TestTemplate public void testWriteSchemaEvolutionRemove() throws Exception { // Create table with original schema String tableName = "write_schema_evolution_remove"; @@ -1226,7 +1268,7 @@ public void testWriteSchemaEvolutionRemove() throws Exception { .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) .load(posDeletesTableName); - Assert.assertEquals(1, scanDF.javaRDD().getNumPartitions()); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); scanDF .writeTo(posDeletesTableName) .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) @@ -1251,18 +1293,22 @@ public void testWriteSchemaEvolutionRemove() throws Exception { }); StructLikeSet expectedA = expected(tab, expectedDeletesA, partitionA, null); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a'", NON_PATH_COLS); - Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = expected(tab, deletesC.first(), partitionC, null); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c'", NON_PATH_COLS); - Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); + assertThat(actualC) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedC); dropTable(tableName); } - @Test + @TestTemplate public void testNormalWritesNotAllowed() throws IOException { String tableName = "test_normal_write_not_allowed"; PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); @@ -1278,10 +1324,9 @@ public void testNormalWritesNotAllowed() throws IOException { Dataset scanDF = spark.read().format("iceberg").load(posDeletesTableName); - Assert.assertThrows( - "position_deletes table can only be written by RewriteDeleteFiles", - IllegalArgumentException.class, - () -> scanDF.writeTo(posDeletesTableName).append()); + assertThatThrownBy(() -> scanDF.writeTo(posDeletesTableName).append()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Can only write to " + posDeletesTableName + " via actions"); dropTable(tableName); } @@ -1457,7 +1502,10 @@ private DataFile dataFile(Table tab, Object[] partDataValues, Object[] partField TestHelpers.Row partitionInfo = TestHelpers.Row.of(partFieldValues); return FileHelpers.writeDataFile( - tab, Files.localOutput(temp.newFile()), partitionInfo, records); + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + partitionInfo, + records); } private Pair>, DeleteFile> deleteFile( @@ -1505,7 +1553,10 @@ private Pair>, DeleteFile> deleteFile( DeleteFile deleteFile = FileHelpers.writePosDeleteFile( - tab, Files.localOutput(temp.newFile()), partitionInfo, deletes); + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + partitionInfo, + deletes); return Pair.of(deletes, deleteFile); } @@ -1529,14 +1580,16 @@ private void commit( Set addedFiles = rewriteCoordinator.fetchNewFiles(posDeletesTable, fileSetID); // Assert new files and old files are equal in number but different in paths - Assert.assertEquals(expectedSourceFiles, rewrittenFiles.size()); - Assert.assertEquals(expectedTargetFiles, addedFiles.size()); + assertThat(rewrittenFiles).hasSize(expectedSourceFiles); + assertThat(addedFiles).hasSize(expectedTargetFiles); List sortedAddedFiles = addedFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); List sortedRewrittenFiles = rewrittenFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); - Assert.assertNotEquals("Lists should not be the same", sortedAddedFiles, sortedRewrittenFiles); + assertThat(sortedRewrittenFiles) + .as("Lists should not be the same") + .isNotEqualTo(sortedAddedFiles); baseTab .newRewrite() diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java index 81a46cd68122..5f59c8eef4ba 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java @@ -24,8 +24,12 @@ import static org.assertj.core.api.Assertions.within; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Schema; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; @@ -36,23 +40,21 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestReadProjection { - final String format; - - TestReadProjection(String format) { - this.format = format; - } + @Parameter(index = 0) + protected FileFormat format; protected abstract Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir protected Path temp; - @Test + @TestTemplate public void testFullProjection() throws Exception { Schema schema = new Schema( @@ -75,7 +77,7 @@ public void testFullProjection() throws Exception { assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); } - @Test + @TestTemplate public void testReorderedFullProjection() throws Exception { // Assume.assumeTrue( // "Spark's Parquet read support does not support reordered columns", @@ -103,7 +105,7 @@ public void testReorderedFullProjection() throws Exception { assertThat(projected.get(1)).as("Should contain the correct 1 value").isEqualTo(34L); } - @Test + @TestTemplate public void testReorderedProjection() throws Exception { // Assume.assumeTrue( // "Spark's Parquet read support does not support reordered columns", @@ -133,7 +135,7 @@ public void testReorderedProjection() throws Exception { assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); } - @Test + @TestTemplate public void testEmptyProjection() throws Exception { Schema schema = new Schema( @@ -151,7 +153,7 @@ public void testEmptyProjection() throws Exception { assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); } - @Test + @TestTemplate public void testBasicProjection() throws Exception { Schema writeSchema = new Schema( @@ -180,7 +182,7 @@ public void testBasicProjection() throws Exception { assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); } - @Test + @TestTemplate public void testRename() throws Exception { Schema writeSchema = new Schema( @@ -206,7 +208,7 @@ public void testRename() throws Exception { assertThat(cmp).as("Should contain the correct data/renamed value").isEqualTo(0); } - @Test + @TestTemplate public void testNestedStructProjection() throws Exception { Schema writeSchema = new Schema( @@ -281,7 +283,7 @@ public void testNestedStructProjection() throws Exception { .isCloseTo(-1.539054f, within(0.000001f)); } - @Test + @TestTemplate public void testMapProjection() throws IOException { Schema writeSchema = new Schema( @@ -339,7 +341,7 @@ public void testMapProjection() throws IOException { return stringMap; } - @Test + @TestTemplate public void testMapOfStructsProjection() throws IOException { Schema writeSchema = new Schema( @@ -463,7 +465,7 @@ public void testMapOfStructsProjection() throws IOException { assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); } - @Test + @TestTemplate public void testListProjection() throws IOException { Schema writeSchema = new Schema( @@ -496,7 +498,7 @@ public void testListProjection() throws IOException { assertThat(projected.getField("values")).as("Should project entire list").isEqualTo(values); } - @Test + @TestTemplate @SuppressWarnings("unchecked") public void testListOfStructsProjection() throws IOException { Schema writeSchema = diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java index 4b57dcd3c891..55fd2cefe2e6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java @@ -18,33 +18,28 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; -import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; - -public class TestRequiredDistributionAndOrdering extends SparkCatalogTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; - public TestRequiredDistributionAndOrdering( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +public class TestRequiredDistributionAndOrdering extends CatalogTestBase { - @After + @AfterEach public void dropTestTable() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testDefaultLocalSort() throws NoSuchTableException { sql( "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " @@ -73,7 +68,7 @@ public void testDefaultLocalSort() throws NoSuchTableException { sql("SELECT count(*) FROM %s", tableName)); } - @Test + @TestTemplate public void testPartitionColumnsArePrependedForRangeDistribution() throws NoSuchTableException { sql( "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " @@ -109,7 +104,7 @@ public void testPartitionColumnsArePrependedForRangeDistribution() throws NoSuch sql("SELECT count(*) FROM %s", tableName)); } - @Test + @TestTemplate public void testSortOrderIncludesPartitionColumns() throws NoSuchTableException { sql( "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " @@ -141,7 +136,7 @@ public void testSortOrderIncludesPartitionColumns() throws NoSuchTableException sql("SELECT count(*) FROM %s", tableName)); } - @Test + @TestTemplate public void testDisabledDistributionAndOrdering() { sql( "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " @@ -162,7 +157,7 @@ public void testDisabledDistributionAndOrdering() { Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail if ordering is disabled - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> inputDF .writeTo(tableName) @@ -176,7 +171,7 @@ public void testDisabledDistributionAndOrdering() { + "and by partition within each spec. Either cluster the incoming records or switch to fanout writers."); } - @Test + @TestTemplate public void testHashDistribution() throws NoSuchTableException { sql( "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " @@ -212,7 +207,7 @@ public void testHashDistribution() throws NoSuchTableException { sql("SELECT count(*) FROM %s", tableName)); } - @Test + @TestTemplate public void testSortBucketTransformsWithoutExtensions() throws NoSuchTableException { sql( "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " @@ -238,7 +233,7 @@ public void testSortBucketTransformsWithoutExtensions() throws NoSuchTableExcept assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); } - @Test + @TestTemplate public void testRangeDistributionWithQuotedColumnsNames() throws NoSuchTableException { sql( "CREATE TABLE %s (c1 INT, c2 STRING, `c.3` STRING) " @@ -274,7 +269,7 @@ public void testRangeDistributionWithQuotedColumnsNames() throws NoSuchTableExce sql("SELECT count(*) FROM %s", tableName)); } - @Test + @TestTemplate public void testHashDistributionWithQuotedColumnsNames() throws NoSuchTableException { sql( "CREATE TABLE %s (c1 INT, c2 STRING, `c``3` STRING) " diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java index edd4cdf08360..21085b49df23 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.PlanningMode.LOCAL; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.io.UncheckedIOException; @@ -29,6 +30,9 @@ import java.util.Set; import org.apache.commons.lang3.StringUtils; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; @@ -36,38 +40,47 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestRuntimeFiltering extends SparkTestBaseWithCatalog { - - @Parameterized.Parameters(name = "planningMode = {0}") - public static Object[] parameters() { - return new Object[] {LOCAL, DISTRIBUTED}; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRuntimeFiltering extends TestBaseWithCatalog { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, planningMode = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + LOCAL + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + DISTRIBUTED + } + }; } - private final PlanningMode planningMode; - - public TestRuntimeFiltering(PlanningMode planningMode) { - this.planningMode = planningMode; - } + @Parameter(index = 3) + private PlanningMode planningMode; - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS dim"); } - @Test + @TestTemplate public void testIdentityPartitionedTable() throws NoSuchTableException { sql( "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " @@ -106,7 +119,7 @@ public void testIdentityPartitionedTable() throws NoSuchTableException { sql(query)); } - @Test + @TestTemplate public void testBucketedTable() throws NoSuchTableException { sql( "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " @@ -145,7 +158,7 @@ public void testBucketedTable() throws NoSuchTableException { sql(query)); } - @Test + @TestTemplate public void testRenamedSourceColumnTable() throws NoSuchTableException { sql( "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " @@ -186,7 +199,7 @@ public void testRenamedSourceColumnTable() throws NoSuchTableException { sql(query)); } - @Test + @TestTemplate public void testMultipleRuntimeFilters() throws NoSuchTableException { sql( "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " @@ -229,7 +242,7 @@ public void testMultipleRuntimeFilters() throws NoSuchTableException { sql(query)); } - @Test + @TestTemplate public void testCaseSensitivityOfRuntimeFilters() throws NoSuchTableException { sql( "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " @@ -273,7 +286,7 @@ public void testCaseSensitivityOfRuntimeFilters() throws NoSuchTableException { sql(caseInsensitiveQuery)); } - @Test + @TestTemplate public void testBucketedTableWithMultipleSpecs() throws NoSuchTableException { sql( "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) USING iceberg", @@ -325,7 +338,7 @@ public void testBucketedTableWithMultipleSpecs() throws NoSuchTableException { sql(query)); } - @Test + @TestTemplate public void testSourceColumnWithDots() throws NoSuchTableException { sql( "CREATE TABLE %s (`i.d` BIGINT, data STRING, date DATE, ts TIMESTAMP) " @@ -369,7 +382,7 @@ public void testSourceColumnWithDots() throws NoSuchTableException { sql(query)); } - @Test + @TestTemplate public void testSourceColumnWithBackticks() throws NoSuchTableException { sql( "CREATE TABLE %s (`i``d` BIGINT, data STRING, date DATE, ts TIMESTAMP) " @@ -410,7 +423,7 @@ public void testSourceColumnWithBackticks() throws NoSuchTableException { sql(query)); } - @Test + @TestTemplate public void testUnpartitionedTable() throws NoSuchTableException { sql( "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) USING iceberg", @@ -458,7 +471,7 @@ private void assertQueryContainsRuntimeFilters( List output = spark.sql("EXPLAIN EXTENDED " + query).collectAsList(); String plan = output.get(0).getString(0); int actualFilterCount = StringUtils.countMatches(plan, "dynamicpruningexpression"); - Assert.assertEquals(errorMessage, expectedFilterCount, actualFilterCount); + assertThat(actualFilterCount).as(errorMessage).isEqualTo(expectedFilterCount); } // delete files that don't match the filter to ensure dynamic filtering works and only required @@ -490,9 +503,8 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo throw new UncheckedIOException(e); } - Assert.assertEquals( - "Deleted unexpected number of files", - expectedDeletedFileCount, - deletedFileLocations.size()); + assertThat(deletedFileLocations) + .as("Deleted unexpected number of files") + .hasSize(expectedDeletedFileCount); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java index 645afd4542e4..a7334a580ca6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java @@ -21,20 +21,24 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PlanningMode; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -44,22 +48,29 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.SparkSession; -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 org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotSelection { - @Parameterized.Parameters(name = "planningMode = {0}") + @Parameters(name = "properties = {0}") public static Object[] parameters() { - return new Object[] {LOCAL, DISTRIBUTED}; + return new Object[][] { + { + ImmutableMap.of( + TableProperties.DATA_PLANNING_MODE, LOCAL.modeName(), + TableProperties.DELETE_PLANNING_MODE, LOCAL.modeName()) + }, + { + ImmutableMap.of( + TableProperties.DATA_PLANNING_MODE, DISTRIBUTED.modeName(), + TableProperties.DELETE_PLANNING_MODE, DISTRIBUTED.modeName()) + } + }; } private static final Configuration CONF = new Configuration(); @@ -67,34 +78,28 @@ public static Object[] parameters() { new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private static SparkSession spark = null; - private final Map properties; - - public TestSnapshotSelection(PlanningMode planningMode) { - this.properties = - ImmutableMap.of( - TableProperties.DATA_PLANNING_MODE, planningMode.modeName(), - TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()); - } + @Parameter(index = 0) + private Map properties; - @BeforeClass + @BeforeAll public static void startSpark() { TestSnapshotSelection.spark = SparkSession.builder().master("local[2]").getOrCreate(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestSnapshotSelection.spark; TestSnapshotSelection.spark = null; currentSpark.stop(); } - @Test + @TestTemplate public void testSnapshotSelectionById() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -114,7 +119,7 @@ public void testSnapshotSelectionById() throws IOException { Dataset secondDf = spark.createDataFrame(secondBatchRecords, SimpleRecord.class); secondDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); - Assert.assertEquals("Expected 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Expected 2 snapshots").hasSize(2); // verify records in the current snapshot Dataset currentSnapshotResult = spark.read().format("iceberg").load(tableLocation); @@ -123,8 +128,9 @@ public void testSnapshotSelectionById() throws IOException { List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(firstBatchRecords); expectedRecords.addAll(secondBatchRecords); - Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, currentSnapshotRecords); + assertThat(currentSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); // verify records in the previous snapshot Snapshot currentSnapshot = table.currentSnapshot(); @@ -133,13 +139,14 @@ public void testSnapshotSelectionById() throws IOException { spark.read().format("iceberg").option("snapshot-id", parentSnapshotId).load(tableLocation); List previousSnapshotRecords = previousSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals( - "Previous snapshot rows should match", firstBatchRecords, previousSnapshotRecords); + assertThat(previousSnapshotRecords) + .as("Previous snapshot rows should match") + .isEqualTo(firstBatchRecords); } - @Test + @TestTemplate public void testSnapshotSelectionByTimestamp() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -162,7 +169,7 @@ public void testSnapshotSelectionByTimestamp() throws IOException { Dataset secondDf = spark.createDataFrame(secondBatchRecords, SimpleRecord.class); secondDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); - Assert.assertEquals("Expected 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Expected 2 snapshots").hasSize(2); // verify records in the current snapshot Dataset currentSnapshotResult = spark.read().format("iceberg").load(tableLocation); @@ -171,8 +178,9 @@ public void testSnapshotSelectionByTimestamp() throws IOException { List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(firstBatchRecords); expectedRecords.addAll(secondBatchRecords); - Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, currentSnapshotRecords); + assertThat(currentSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); // verify records in the previous snapshot Dataset previousSnapshotResult = @@ -183,13 +191,14 @@ public void testSnapshotSelectionByTimestamp() throws IOException { .load(tableLocation); List previousSnapshotRecords = previousSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals( - "Previous snapshot rows should match", firstBatchRecords, previousSnapshotRecords); + assertThat(previousSnapshotRecords) + .as("Previous snapshot rows should match") + .isEqualTo(firstBatchRecords); } - @Test + @TestTemplate public void testSnapshotSelectionByInvalidSnapshotId() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -197,21 +206,21 @@ public void testSnapshotSelectionByInvalidSnapshotId() throws IOException { Dataset df = spark.read().format("iceberg").option("snapshot-id", -10).load(tableLocation); - Assertions.assertThatThrownBy(df::collectAsList) + assertThatThrownBy(df::collectAsList) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find snapshot with ID -10"); } - @Test + @TestTemplate public void testSnapshotSelectionByInvalidTimestamp() throws IOException { long timestamp = System.currentTimeMillis(); - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); tables.create(SCHEMA, spec, properties, tableLocation); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -222,9 +231,9 @@ public void testSnapshotSelectionByInvalidTimestamp() throws IOException { .hasMessageContaining("Cannot find a snapshot older than"); } - @Test + @TestTemplate public void testSnapshotSelectionBySnapshotIdAndTimestamp() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -239,7 +248,7 @@ public void testSnapshotSelectionBySnapshotIdAndTimestamp() throws IOException { long timestamp = System.currentTimeMillis(); long snapshotId = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -254,9 +263,9 @@ public void testSnapshotSelectionBySnapshotIdAndTimestamp() throws IOException { .hasMessageContaining("tag"); } - @Test + @TestTemplate public void testSnapshotSelectionByTag() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -285,13 +294,14 @@ public void testSnapshotSelectionByTag() throws IOException { currentSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(firstBatchRecords); - Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, currentSnapshotRecords); + assertThat(currentSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); } - @Test + @TestTemplate public void testSnapshotSelectionByBranch() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -320,13 +330,14 @@ public void testSnapshotSelectionByBranch() throws IOException { currentSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(firstBatchRecords); - Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, currentSnapshotRecords); + assertThat(currentSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); } - @Test + @TestTemplate public void testSnapshotSelectionByBranchAndTagFails() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -342,7 +353,7 @@ public void testSnapshotSelectionByBranchAndTagFails() throws IOException { table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -355,9 +366,9 @@ public void testSnapshotSelectionByBranchAndTagFails() throws IOException { .hasMessageStartingWith("Can specify only one of snapshot-id"); } - @Test + @TestTemplate public void testSnapshotSelectionByTimestampAndBranchOrTagFails() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -373,7 +384,7 @@ public void testSnapshotSelectionByTimestampAndBranchOrTagFails() throws IOExcep table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -385,7 +396,7 @@ public void testSnapshotSelectionByTimestampAndBranchOrTagFails() throws IOExcep .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Can specify only one of snapshot-id"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -398,9 +409,9 @@ public void testSnapshotSelectionByTimestampAndBranchOrTagFails() throws IOExcep .hasMessageStartingWith("Can specify only one of snapshot-id"); } - @Test + @TestTemplate public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -421,14 +432,15 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { branchSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(firstBatchRecords); - Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, branchSnapshotRecords); + assertThat(branchSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); // Deleting a column to indicate schema change table.updateSchema().deleteColumn("data").commit(); // The data should not have the deleted column - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -441,7 +453,7 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { // re-introducing the column should not let the data re-appear table.updateSchema().addColumn("data", Types.StringType.get()).commit(); - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -454,9 +466,9 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { new SimpleRecord(1, null), new SimpleRecord(2, null), new SimpleRecord(3, null)); } - @Test + @TestTemplate public void testWritingToBranchAfterSchemaChange() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -477,13 +489,14 @@ public void testWritingToBranchAfterSchemaChange() throws IOException { branchSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(firstBatchRecords); - Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, branchSnapshotRecords); + assertThat(branchSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); // Deleting and add a new column of the same type to indicate schema change table.updateSchema().deleteColumn("data").addColumn("zip", Types.IntegerType.get()).commit(); - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -514,7 +527,7 @@ public void testWritingToBranchAfterSchemaChange() throws IOException { .mode("append") .save(tableLocation); - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -527,9 +540,9 @@ public void testWritingToBranchAfterSchemaChange() throws IOException { .containsAll(records); } - @Test + @TestTemplate public void testSnapshotSelectionByTagWithSchemaChange() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -551,7 +564,9 @@ public void testSnapshotSelectionByTagWithSchemaChange() throws IOException { spark.read().format("iceberg").option("tag", "tag").load(tableLocation); List tagSnapshotRecords = tagSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Current snapshot rows should match", expectedRecords, tagSnapshotRecords); + assertThat(tagSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); // Deleting a column to indicate schema change table.updateSchema().deleteColumn("data").commit(); @@ -564,7 +579,8 @@ public void testSnapshotSelectionByTagWithSchemaChange() throws IOException { .orderBy("id") .as(Encoders.bean(SimpleRecord.class)) .collectAsList(); - Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, deletedColumnTagSnapshotRecords); + assertThat(deletedColumnTagSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java index 607f1d45ba3a..7ff507ed0694 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java @@ -18,25 +18,25 @@ */ package org.apache.iceberg.spark.source; -import java.util.Map; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.KryoHelpers; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalog; -import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkSessionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; -public class TestSparkCatalogHadoopOverrides extends SparkCatalogTestBase { +public class TestSparkCatalogHadoopOverrides extends CatalogTestBase { private static final String configToOverride = "fs.s3a.buffer.dir"; // prepend "hadoop." so that the test base formats SQLConf correctly @@ -44,7 +44,7 @@ public class TestSparkCatalogHadoopOverrides extends SparkCatalogTestBase { private static final String hadoopPrefixedConfigToOverride = "hadoop." + configToOverride; private static final String configOverrideValue = "/tmp-overridden"; - @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { return new Object[][] { { @@ -77,41 +77,36 @@ public static Object[][] parameters() { }; } - public TestSparkCatalogHadoopOverrides( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Before + @BeforeEach public void createTable() { sql("CREATE TABLE IF NOT EXISTS %s (id bigint) USING iceberg", tableName(tableIdent.name())); } - @After + @AfterEach public void dropTable() { sql("DROP TABLE IF EXISTS %s", tableName(tableIdent.name())); } - @Test + @TestTemplate public void testTableFromCatalogHasOverrides() throws Exception { Table table = getIcebergTableFromSparkCatalog(); Configuration conf = ((Configurable) table.io()).getConf(); String actualCatalogOverride = conf.get(configToOverride, "/whammies"); - Assert.assertEquals( - "Iceberg tables from spark should have the overridden hadoop configurations from the spark config", - configOverrideValue, - actualCatalogOverride); + assertThat(actualCatalogOverride) + .as( + "Iceberg tables from spark should have the overridden hadoop configurations from the spark config") + .isEqualTo(configOverrideValue); } - @Test + @TestTemplate public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception { Table table = getIcebergTableFromSparkCatalog(); Configuration originalConf = ((Configurable) table.io()).getConf(); String actualCatalogOverride = originalConf.get(configToOverride, "/whammies"); - Assert.assertEquals( - "Iceberg tables from spark should have the overridden hadoop configurations from the spark config", - configOverrideValue, - actualCatalogOverride); + assertThat(actualCatalogOverride) + .as( + "Iceberg tables from spark should have the overridden hadoop configurations from the spark config") + .isEqualTo(configOverrideValue); // Now convert to SerializableTable and ensure overridden property is still present. Table serializableTable = SerializableTableWithSize.copyOf(table); @@ -119,19 +114,19 @@ public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception KryoHelpers.roundTripSerialize(SerializableTableWithSize.copyOf(table)); Configuration configFromKryoSerde = ((Configurable) kryoSerializedTable.io()).getConf(); String kryoSerializedCatalogOverride = configFromKryoSerde.get(configToOverride, "/whammies"); - Assert.assertEquals( - "Tables serialized with Kryo serialization should retain overridden hadoop configuration properties", - configOverrideValue, - kryoSerializedCatalogOverride); + assertThat(kryoSerializedCatalogOverride) + .as( + "Tables serialized with Kryo serialization should retain overridden hadoop configuration properties") + .isEqualTo(configOverrideValue); // Do the same for Java based serde Table javaSerializedTable = TestHelpers.roundTripSerialize(serializableTable); Configuration configFromJavaSerde = ((Configurable) javaSerializedTable.io()).getConf(); String javaSerializedCatalogOverride = configFromJavaSerde.get(configToOverride, "/whammies"); - Assert.assertEquals( - "Tables serialized with Java serialization should retain overridden hadoop configuration properties", - configOverrideValue, - javaSerializedCatalogOverride); + assertThat(javaSerializedCatalogOverride) + .as( + "Tables serialized with Java serialization should retain overridden hadoop configuration properties") + .isEqualTo(configOverrideValue); } @SuppressWarnings("ThrowSpecificity") diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index d6a235674d63..fb2b312bed97 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -20,12 +20,16 @@ import static org.apache.iceberg.TableProperties.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -34,6 +38,9 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotRef; @@ -52,65 +59,61 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.AfterAll; +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; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestSparkDataWrite { private static final Configuration CONF = new Configuration(); - private final FileFormat format; - private final String branch; + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private String branch; + private static SparkSession spark = null; private static final Schema SCHEMA = new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @Parameterized.Parameters(name = "format = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[] { - new Object[] {"parquet", null}, - new Object[] {"parquet", "main"}, - new Object[] {"parquet", "testBranch"}, - new Object[] {"avro", null}, - new Object[] {"orc", "testBranch"} + @Parameters(name = "format = {0}, branch = {1}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.PARQUET, null}, + new Object[] {FileFormat.PARQUET, "main"}, + new Object[] {FileFormat.PARQUET, "testBranch"}, + new Object[] {FileFormat.AVRO, null}, + new Object[] {FileFormat.ORC, "testBranch"} }; } - @BeforeClass + @BeforeAll public static void startSpark() { TestSparkDataWrite.spark = SparkSession.builder().master("local[2]").getOrCreate(); } - @Parameterized.AfterParam - public static void clearSourceCache() { + @AfterEach + public void clearSourceCache() { ManualSource.clearTables(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestSparkDataWrite.spark; TestSparkDataWrite.spark = null; currentSpark.stop(); } - public TestSparkDataWrite(String format, String branch) { - this.format = FileFormat.fromString(format); - this.branch = branch; - } - - @Test + @TestTemplate public void testBasicWrite() throws IOException { - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -138,31 +141,31 @@ public void testBasicWrite() throws IOException { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); for (ManifestFile manifest : SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io())) { for (DataFile file : ManifestFiles.read(manifest, table.io())) { // TODO: avro not support split if (!format.equals(FileFormat.AVRO)) { - Assert.assertNotNull("Split offsets not present", file.splitOffsets()); + assertThat(file.splitOffsets()).as("Split offsets not present").isNotNull(); } - Assert.assertEquals("Should have reported record count as 1", 1, file.recordCount()); + assertThat(file.recordCount()).as("Should have reported record count as 1").isEqualTo(1); // TODO: append more metric info if (format.equals(FileFormat.PARQUET)) { - Assert.assertNotNull("Column sizes metric not present", file.columnSizes()); - Assert.assertNotNull("Counts metric not present", file.valueCounts()); - Assert.assertNotNull("Null value counts metric not present", file.nullValueCounts()); - Assert.assertNotNull("Lower bounds metric not present", file.lowerBounds()); - Assert.assertNotNull("Upper bounds metric not present", file.upperBounds()); + assertThat(file.columnSizes()).as("Column sizes metric not present").isNotNull(); + assertThat(file.valueCounts()).as("Counts metric not present").isNotNull(); + assertThat(file.nullValueCounts()).as("Null value counts metric not present").isNotNull(); + assertThat(file.lowerBounds()).as("Lower bounds metric not present").isNotNull(); + assertThat(file.upperBounds()).as("Upper bounds metric not present").isNotNull(); } } } } - @Test + @TestTemplate public void testAppend() throws IOException { - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -208,13 +211,13 @@ public void testAppend() throws IOException { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testEmptyOverwrite() throws IOException { - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -254,13 +257,13 @@ public void testEmptyOverwrite() throws IOException { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testOverwrite() throws IOException { - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -307,13 +310,13 @@ public void testOverwrite() throws IOException { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testUnpartitionedOverwrite() throws IOException { - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -350,13 +353,13 @@ public void testUnpartitionedOverwrite() throws IOException { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testUnpartitionedCreateWithTargetFileSizeViaTableProperties() throws IOException { - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -390,8 +393,8 @@ public void testUnpartitionedCreateWithTargetFileSizeViaTableProperties() throws List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); List files = Lists.newArrayList(); for (ManifestFile manifest : @@ -401,33 +404,34 @@ public void testUnpartitionedCreateWithTargetFileSizeViaTableProperties() throws } } - Assert.assertEquals("Should have 4 DataFiles", 4, files.size()); - Assert.assertTrue( - "All DataFiles contain 1000 rows", files.stream().allMatch(d -> d.recordCount() == 1000)); + assertThat(files).as("Should have 4 DataFiles").hasSize(4); + assertThat(files.stream()) + .as("All DataFiles contain 1000 rows") + .allMatch(d -> d.recordCount() == 1000); } - @Test + @TestTemplate public void testPartitionedCreateWithTargetFileSizeViaOption() throws IOException { partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType.NONE); } - @Test + @TestTemplate public void testPartitionedFanoutCreateWithTargetFileSizeViaOption() throws IOException { partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType.TABLE); } - @Test + @TestTemplate public void testPartitionedFanoutCreateWithTargetFileSizeViaOption2() throws IOException { partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType.JOB); } - @Test + @TestTemplate public void testWriteProjection() throws IOException { - Assume.assumeTrue( - "Not supported in Spark 3; analysis requires all columns are present", - spark.version().startsWith("2")); + assumeThat(spark.version()) + .as("Not supported in Spark 3; analysis requires all columns are present") + .startsWith("2"); - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -455,17 +459,17 @@ public void testWriteProjection() throws IOException { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testWriteProjectionWithMiddle() throws IOException { - Assume.assumeTrue( - "Not supported in Spark 3; analysis requires all columns are present", - spark.version().startsWith("2")); + assumeThat(spark.version()) + .as("Not supported in Spark 3; analysis requires all columns are present") + .startsWith("2"); - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -500,13 +504,13 @@ public void testWriteProjectionWithMiddle() throws IOException { List actual = result.orderBy("c1").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); } - @Test + @TestTemplate public void testViewsReturnRecentResults() throws IOException { - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -536,8 +540,8 @@ public void testViewsReturnRecentResults() throws IOException { List actual1 = spark.table("tmp").as(Encoders.bean(SimpleRecord.class)).collectAsList(); List expected1 = Lists.newArrayList(new SimpleRecord(1, "a")); - Assert.assertEquals("Number of rows should match", expected1.size(), actual1.size()); - Assert.assertEquals("Result rows should match", expected1, actual1); + assertThat(actual1).as("Number of rows should match").hasSameSizeAs(expected1); + assertThat(actual1).as("Result rows should match").isEqualTo(expected1); df.select("id", "data") .write() @@ -550,13 +554,13 @@ public void testViewsReturnRecentResults() throws IOException { spark.table("tmp").as(Encoders.bean(SimpleRecord.class)).collectAsList(); List expected2 = Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(1, "a")); - Assert.assertEquals("Number of rows should match", expected2.size(), actual2.size()); - Assert.assertEquals("Result rows should match", expected2, actual2); + assertThat(actual2).as("Number of rows should match").hasSameSizeAs(expected2); + assertThat(actual2).as("Result rows should match").isEqualTo(expected2); } public void partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType option) throws IOException { - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); String targetLocation = locationWithBranch(location); @@ -619,8 +623,8 @@ public void partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType opti List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); - Assert.assertEquals("Result rows should match", expected, actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); List files = Lists.newArrayList(); for (ManifestFile manifest : @@ -629,15 +633,15 @@ public void partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType opti files.add(file); } } - - Assert.assertEquals("Should have 8 DataFiles", 8, files.size()); - Assert.assertTrue( - "All DataFiles contain 1000 rows", files.stream().allMatch(d -> d.recordCount() == 1000)); + assertThat(files).as("Should have 8 DataFiles").hasSize(8); + assertThat(files.stream()) + .as("All DataFiles contain 1000 rows") + .allMatch(d -> d.recordCount() == 1000); } - @Test + @TestTemplate public void testCommitUnknownException() throws IOException { - File parent = temp.newFolder(format.toString()); + File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "commitunknown"); String targetLocation = locationWithBranch(location); @@ -689,7 +693,7 @@ public void testCommitUnknownException() throws IOException { ManualSource.setTable(manualTableName, sparkTable); // Although an exception is thrown here, write and commit have succeeded - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> df2.select("id", "data") .sort("data") @@ -705,9 +709,8 @@ public void testCommitUnknownException() throws IOException { Dataset result = spark.read().format("iceberg").load(targetLocation); List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals( - "Number of rows should match", records.size() + records2.size(), actual.size()); - Assertions.assertThat(actual) + assertThat(actual).as("Number of rows should match").hasSize(records.size() + records2.size()); + assertThat(actual) .describedAs("Result rows should match") .containsExactlyInAnyOrder( ImmutableList.builder() diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index b2361c188c17..2924cd5db5b2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -25,14 +25,21 @@ import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.stream.Collectors; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -46,26 +53,22 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; 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.Assume; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; -@RunWith(Parameterized.class) -public class TestSparkMetadataColumns extends SparkTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkMetadataColumns extends TestBase { private static final String TABLE_NAME = "test_table"; private static final Schema SCHEMA = @@ -81,7 +84,7 @@ public class TestSparkMetadataColumns extends SparkTestBase { .addField("zero", 1, "id_zero") .build(); - @Parameterized.Parameters(name = "fileFormat = {0}, vectorized = {1}, formatVersion = {2}") + @Parameters(name = "fileFormat = {0}, vectorized = {1}, formatVersion = {2}") public static Object[][] parameters() { return new Object[][] { {FileFormat.PARQUET, false, 1}, @@ -97,21 +100,20 @@ public static Object[][] parameters() { }; } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - private final FileFormat fileFormat; - private final boolean vectorized; - private final int formatVersion; + @Parameter(index = 0) + private FileFormat fileFormat; - private Table table = null; + @Parameter(index = 1) + private boolean vectorized; - public TestSparkMetadataColumns(FileFormat fileFormat, boolean vectorized, int formatVersion) { - this.fileFormat = fileFormat; - this.vectorized = vectorized; - this.formatVersion = formatVersion; - } + @Parameter(index = 2) + private int formatVersion; + + private Table table = null; - @BeforeClass + @BeforeAll public static void setupSpark() { ImmutableMap config = ImmutableMap.of( @@ -125,17 +127,17 @@ public static void setupSpark() { (key, value) -> spark.conf().set("spark.sql.catalog.spark_catalog." + key, value)); } - @Before + @BeforeEach public void setupTable() throws IOException { createAndInitTable(); } - @After + @AfterEach public void dropTable() { TestTables.clearTables(); } - @Test + @TestTemplate public void testSpecAndPartitionMetadataColumns() { // TODO: support metadata structs in vectorized ORC reads Assume.assumeFalse(fileFormat == FileFormat.ORC && vectorized); @@ -169,7 +171,7 @@ public void testSpecAndPartitionMetadataColumns() { sql("SELECT _spec_id, _partition FROM %s ORDER BY _spec_id", TABLE_NAME)); } - @Test + @TestTemplate public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTableException { Assume.assumeTrue(fileFormat == FileFormat.PARQUET); @@ -187,13 +189,13 @@ public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTable .withColumn("data", lit("ABCDEF")); df.coalesce(1).writeTo(TABLE_NAME).append(); - Assert.assertEquals(200, spark.table(TABLE_NAME).count()); + assertThat(spark.table(TABLE_NAME).count()).isEqualTo(200); List expectedRows = ids.stream().map(this::row).collect(Collectors.toList()); assertEquals("Rows must match", expectedRows, sql("SELECT _pos FROM %s", TABLE_NAME)); } - @Test + @TestTemplate public void testPositionMetadataColumnWithMultipleBatches() throws NoSuchTableException { Assume.assumeTrue(fileFormat == FileFormat.PARQUET); @@ -211,25 +213,25 @@ public void testPositionMetadataColumnWithMultipleBatches() throws NoSuchTableEx .withColumn("data", lit("ABCDEF")); df.coalesce(1).writeTo(TABLE_NAME).append(); - Assert.assertEquals(7500, spark.table(TABLE_NAME).count()); + assertThat(spark.table(TABLE_NAME).count()).isEqualTo(7500); List expectedRows = ids.stream().map(this::row).collect(Collectors.toList()); assertEquals("Rows must match", expectedRows, sql("SELECT _pos FROM %s", TABLE_NAME)); } - @Test + @TestTemplate public void testPartitionMetadataColumnWithUnknownTransforms() { // replace the table spec to include an unknown transform TableOperations ops = ((HasTableOperations) table).operations(); TableMetadata base = ops.current(); ops.commit(base, base.updatePartitionSpec(UNKNOWN_SPEC)); - Assertions.assertThatThrownBy(() -> sql("SELECT _partition FROM %s", TABLE_NAME)) + assertThatThrownBy(() -> sql("SELECT _partition FROM %s", TABLE_NAME)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot build table partition type, unknown transforms: [zero]"); } - @Test + @TestTemplate public void testConflictingColumns() { table .updateSchema() @@ -244,7 +246,7 @@ public void testConflictingColumns() { ImmutableList.of(row(1L, "a1")), sql("SELECT id, category FROM %s", TABLE_NAME)); - Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s", TABLE_NAME)) + assertThatThrownBy(() -> sql("SELECT * FROM %s", TABLE_NAME)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith( "Table column names conflict with names reserved for Iceberg metadata columns: [_spec_id, _file]."); @@ -281,6 +283,12 @@ private void createAndInitTable() throws IOException { !vectorized, "File format %s does not support vectorized reads", fileFormat); } - this.table = TestTables.create(temp.newFolder(), TABLE_NAME, SCHEMA, SPEC, properties); + this.table = + TestTables.create( + Files.createTempDirectory(temp, "junit").toFile(), + TABLE_NAME, + SCHEMA, + SPEC, + properties); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java index 3a4b235c46e5..99a327402d97 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java @@ -23,6 +23,7 @@ import static org.apache.iceberg.PlanningMode.LOCAL; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -32,6 +33,9 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.Schema; @@ -51,40 +55,33 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class TestSparkReadProjection extends TestReadProjection { private static SparkSession spark = null; - @Parameterized.Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") + @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") public static Object[][] parameters() { return new Object[][] { - {"parquet", false, LOCAL}, - {"parquet", true, DISTRIBUTED}, - {"avro", false, LOCAL}, - {"orc", false, DISTRIBUTED}, - {"orc", true, LOCAL} + {FileFormat.PARQUET, false, LOCAL}, + {FileFormat.PARQUET, true, DISTRIBUTED}, + {FileFormat.AVRO, false, LOCAL}, + {FileFormat.ORC, false, DISTRIBUTED}, + {FileFormat.ORC, true, LOCAL} }; } - private final FileFormat format; - private final boolean vectorized; - private final PlanningMode planningMode; + @Parameter(index = 1) + private boolean vectorized; - public TestSparkReadProjection(String format, boolean vectorized, PlanningMode planningMode) { - super(format); - this.format = FileFormat.fromString(format); - this.vectorized = vectorized; - this.planningMode = planningMode; - } + @Parameter(index = 2) + private PlanningMode planningMode; - @BeforeClass + @BeforeAll public static void startSpark() { TestSparkReadProjection.spark = SparkSession.builder().master("local[2]").getOrCreate(); ImmutableMap config = @@ -100,7 +97,7 @@ public static void startSpark() { (key, value) -> spark.conf().set("spark.sql.catalog.spark_catalog." + key, value)); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestSparkReadProjection.spark; TestSparkReadProjection.spark = null; @@ -110,10 +107,10 @@ public static void stopSpark() { @Override protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException { - File parent = temp.newFolder(desc); + File parent = new File(temp.toFile(), desc); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); - Assert.assertTrue("mkdirs should succeed", dataFolder.mkdirs()); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); File testFile = new File(dataFolder, format.addExtension(UUID.randomUUID().toString())); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java index e5831b76e424..baf7fa8f88a2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java @@ -24,10 +24,13 @@ import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT; +import static org.assertj.core.api.Assertions.assertThat; import java.io.Closeable; +import java.io.File; import java.io.IOException; import java.math.BigDecimal; +import java.nio.file.Path; import java.time.LocalDate; import java.util.List; import java.util.Map; @@ -38,6 +41,9 @@ import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; @@ -62,18 +68,15 @@ import org.apache.iceberg.util.PropertyUtil; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestSparkReaderWithBloomFilter { protected String tableName = null; @@ -84,13 +87,12 @@ public class TestSparkReaderWithBloomFilter { private static TestHiveMetastore metastore = null; protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - protected final boolean vectorized; - protected final boolean useBloomFilter; - public TestSparkReaderWithBloomFilter(boolean vectorized, boolean useBloomFilter) { - this.vectorized = vectorized; - this.useBloomFilter = useBloomFilter; - } + @Parameter(index = 0) + protected boolean vectorized; + + @Parameter(index = 1) + protected boolean useBloomFilter; // Schema passed to create tables public static final Schema SCHEMA = @@ -114,9 +116,9 @@ public TestSparkReaderWithBloomFilter(boolean vectorized, boolean useBloomFilter private static final float FLOAT_BASE = 100000F; private static final String BINARY_PREFIX = "BINARY测试_"; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @Before + @BeforeEach public void writeTestDataFile() throws IOException { this.tableName = "test"; createTable(tableName, SCHEMA); @@ -151,22 +153,26 @@ public void writeTestDataFile() throws IOException { new BigDecimal(String.valueOf(99.99))))); } - this.dataFile = writeDataFile(Files.localOutput(temp.newFile()), Row.of(0), records); + this.dataFile = + writeDataFile( + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + Row.of(0), + records); table.newAppend().appendFile(dataFile).commit(); } - @After + @AfterEach public void cleanup() throws IOException { dropTable("test"); } - @Parameterized.Parameters(name = "vectorized = {0}, useBloomFilter = {1}") + @Parameters(name = "vectorized = {0}, useBloomFilter = {1}") public static Object[][] parameters() { return new Object[][] {{false, false}, {true, false}, {false, true}, {true, true}}; } - @BeforeClass + @BeforeAll public static void startMetastoreAndSpark() { metastore = new TestHiveMetastore(); metastore.start(); @@ -191,7 +197,7 @@ public static void startMetastoreAndSpark() { } } - @AfterClass + @AfterAll public static void stopMetastoreAndSpark() throws Exception { catalog = null; metastore.stop(); @@ -334,7 +340,7 @@ private FileFormat defaultFormat(Map properties) { return FileFormat.fromString(formatString); } - @Test + @TestTemplate public void testReadWithFilter() { Dataset df = spark @@ -349,9 +355,8 @@ public void testReadWithFilter() { Record record = SparkValueConverter.convert(table.schema(), df.collectAsList().get(0)); - Assert.assertEquals("Table should contain 1 row", 1, df.collectAsList().size()); - - Assert.assertEquals("Table should contain expected rows", record.get(0), 30); + assertThat(df.collectAsList()).as("Table should contain 1 row").hasSize(1); + assertThat(record.get(0)).as("Table should contain expected rows").isEqualTo(30); df = spark @@ -366,8 +371,7 @@ public void testReadWithFilter() { record = SparkValueConverter.convert(table.schema(), df.collectAsList().get(0)); - Assert.assertEquals("Table should contain 1 row", 1, df.collectAsList().size()); - - Assert.assertEquals("Table should contain expected rows", record.get(0), 250); + assertThat(df.collectAsList()).as("Table should contain 1 row").hasSize(1); + assertThat(record.get(0)).as("Table should contain expected rows").isEqualTo(250); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index 78d169bf7340..d539b0123951 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -26,12 +26,17 @@ import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToYearOrdinal; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; +import static org.assertj.core.api.Assertions.assertThat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.spark.functions.BucketFunction; import org.apache.iceberg.spark.functions.DaysFunction; import org.apache.iceberg.spark.functions.HoursFunction; @@ -57,39 +62,52 @@ import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestSparkScan extends SparkTestBaseWithCatalog { - - private final String format; - - @Parameterized.Parameters(name = "format = {0}") - public static Object[] parameters() { - return new Object[] {"parquet", "avro", "orc"}; - } - - public TestSparkScan(String format) { - this.format = format; - } - - @Before +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; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkScan extends TestBaseWithCatalog { + + @Parameter(index = 3) + private String format; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, format = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + "parquet" + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + "avro" + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + "orc" + } + }; + } + + @BeforeEach public void useCatalog() { sql("USE %s", catalogName); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testEstimatedRowCount() throws NoSuchTableException { sql( "CREATE TABLE %s (id BIGINT, date DATE) USING iceberg TBLPROPERTIES('%s' = '%s')", @@ -109,10 +127,10 @@ public void testEstimatedRowCount() throws NoSuchTableException { SparkScan scan = (SparkScan) scanBuilder.build(); Statistics stats = scan.estimateStatistics(); - Assert.assertEquals(10000L, stats.numRows().getAsLong()); + assertThat(stats.numRows().getAsLong()).isEqualTo(10000L); } - @Test + @TestTemplate public void testUnpartitionedYears() throws Exception { createUnpartitionedTable(spark, tableName); @@ -128,7 +146,7 @@ public void testUnpartitionedYears() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT Equal builder = scanBuilder(); @@ -137,10 +155,10 @@ public void testUnpartitionedYears() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedYears() throws Exception { createPartitionedTable(spark, tableName, "years(ts)"); @@ -156,7 +174,7 @@ public void testPartitionedYears() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); // NOT Equal builder = scanBuilder(); @@ -165,10 +183,10 @@ public void testPartitionedYears() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); } - @Test + @TestTemplate public void testUnpartitionedMonths() throws Exception { createUnpartitionedTable(spark, tableName); @@ -185,7 +203,7 @@ public void testUnpartitionedMonths() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT GT builder = scanBuilder(); @@ -194,10 +212,10 @@ public void testUnpartitionedMonths() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedMonths() throws Exception { createPartitionedTable(spark, tableName, "months(ts)"); @@ -214,7 +232,7 @@ public void testPartitionedMonths() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); // NOT GT builder = scanBuilder(); @@ -223,10 +241,10 @@ public void testPartitionedMonths() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); } - @Test + @TestTemplate public void testUnpartitionedDays() throws Exception { createUnpartitionedTable(spark, tableName); @@ -242,7 +260,7 @@ public void testUnpartitionedDays() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT LT builder = scanBuilder(); @@ -251,10 +269,10 @@ public void testUnpartitionedDays() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedDays() throws Exception { createPartitionedTable(spark, tableName, "days(ts)"); @@ -270,7 +288,7 @@ public void testPartitionedDays() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); // NOT LT builder = scanBuilder(); @@ -279,10 +297,10 @@ public void testPartitionedDays() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); } - @Test + @TestTemplate public void testUnpartitionedHours() throws Exception { createUnpartitionedTable(spark, tableName); @@ -298,7 +316,7 @@ public void testUnpartitionedHours() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT GTEQ builder = scanBuilder(); @@ -307,10 +325,10 @@ public void testUnpartitionedHours() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedHours() throws Exception { createPartitionedTable(spark, tableName, "hours(ts)"); @@ -326,7 +344,7 @@ public void testPartitionedHours() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(8); + assertThat(scan.planInputPartitions().length).isEqualTo(8); // NOT GTEQ builder = scanBuilder(); @@ -335,10 +353,10 @@ public void testPartitionedHours() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(2); + assertThat(scan.planInputPartitions().length).isEqualTo(2); } - @Test + @TestTemplate public void testUnpartitionedBucketLong() throws Exception { createUnpartitionedTable(spark, tableName); @@ -350,7 +368,7 @@ public void testUnpartitionedBucketLong() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT GTEQ builder = scanBuilder(); @@ -359,10 +377,10 @@ public void testUnpartitionedBucketLong() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedBucketLong() throws Exception { createPartitionedTable(spark, tableName, "bucket(5, id)"); @@ -374,7 +392,7 @@ public void testPartitionedBucketLong() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(6); + assertThat(scan.planInputPartitions().length).isEqualTo(6); // NOT GTEQ builder = scanBuilder(); @@ -383,10 +401,10 @@ public void testPartitionedBucketLong() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(4); + assertThat(scan.planInputPartitions().length).isEqualTo(4); } - @Test + @TestTemplate public void testUnpartitionedBucketString() throws Exception { createUnpartitionedTable(spark, tableName); @@ -398,7 +416,7 @@ public void testUnpartitionedBucketString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT LTEQ builder = scanBuilder(); @@ -407,10 +425,10 @@ public void testUnpartitionedBucketString() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedBucketString() throws Exception { createPartitionedTable(spark, tableName, "bucket(5, data)"); @@ -422,7 +440,7 @@ public void testPartitionedBucketString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(6); + assertThat(scan.planInputPartitions().length).isEqualTo(6); // NOT LTEQ builder = scanBuilder(); @@ -431,10 +449,10 @@ public void testPartitionedBucketString() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(4); + assertThat(scan.planInputPartitions().length).isEqualTo(4); } - @Test + @TestTemplate public void testUnpartitionedTruncateString() throws Exception { createUnpartitionedTable(spark, tableName); @@ -446,7 +464,7 @@ public void testUnpartitionedTruncateString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT NotEqual builder = scanBuilder(); @@ -455,10 +473,10 @@ public void testUnpartitionedTruncateString() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedTruncateString() throws Exception { createPartitionedTable(spark, tableName, "truncate(4, data)"); @@ -470,7 +488,7 @@ public void testPartitionedTruncateString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); // NOT NotEqual builder = scanBuilder(); @@ -479,10 +497,10 @@ public void testPartitionedTruncateString() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); } - @Test + @TestTemplate public void testUnpartitionedIsNull() throws Exception { createUnpartitionedTable(spark, tableName); @@ -494,7 +512,7 @@ public void testUnpartitionedIsNull() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT IsNull builder = scanBuilder(); @@ -503,10 +521,10 @@ public void testUnpartitionedIsNull() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedIsNull() throws Exception { createPartitionedTable(spark, tableName, "truncate(4, data)"); @@ -518,7 +536,7 @@ public void testPartitionedIsNull() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(0); + assertThat(scan.planInputPartitions().length).isEqualTo(0); // NOT IsNULL builder = scanBuilder(); @@ -527,10 +545,10 @@ public void testPartitionedIsNull() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testUnpartitionedIsNotNull() throws Exception { createUnpartitionedTable(spark, tableName); @@ -542,7 +560,7 @@ public void testUnpartitionedIsNotNull() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT IsNotNull builder = scanBuilder(); @@ -551,10 +569,10 @@ public void testUnpartitionedIsNotNull() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedIsNotNull() throws Exception { createPartitionedTable(spark, tableName, "truncate(4, data)"); @@ -566,7 +584,7 @@ public void testPartitionedIsNotNull() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT IsNotNULL builder = scanBuilder(); @@ -575,10 +593,10 @@ public void testPartitionedIsNotNull() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(0); + assertThat(scan.planInputPartitions().length).isEqualTo(0); } - @Test + @TestTemplate public void testUnpartitionedAnd() throws Exception { createUnpartitionedTable(spark, tableName); @@ -596,7 +614,7 @@ public void testUnpartitionedAnd() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT (years(ts) = 47 AND bucket(id, 5) >= 2) builder = scanBuilder(); @@ -605,10 +623,10 @@ public void testUnpartitionedAnd() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedAnd() throws Exception { createPartitionedTable(spark, tableName, "years(ts), bucket(5, id)"); @@ -626,7 +644,7 @@ public void testPartitionedAnd() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(1); + assertThat(scan.planInputPartitions().length).isEqualTo(1); // NOT (years(ts) = 47 AND bucket(id, 5) >= 2) builder = scanBuilder(); @@ -635,10 +653,10 @@ public void testPartitionedAnd() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(9); + assertThat(scan.planInputPartitions().length).isEqualTo(9); } - @Test + @TestTemplate public void testUnpartitionedOr() throws Exception { createUnpartitionedTable(spark, tableName); @@ -656,7 +674,7 @@ public void testUnpartitionedOr() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT (years(ts) = 47 OR bucket(id, 5) >= 2) builder = scanBuilder(); @@ -665,10 +683,10 @@ public void testUnpartitionedOr() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } - @Test + @TestTemplate public void testPartitionedOr() throws Exception { createPartitionedTable(spark, tableName, "years(ts), bucket(5, id)"); @@ -686,7 +704,7 @@ public void testPartitionedOr() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(6); + assertThat(scan.planInputPartitions().length).isEqualTo(6); // NOT (years(ts) = 48 OR bucket(id, 5) >= 2) builder = scanBuilder(); @@ -695,7 +713,7 @@ public void testPartitionedOr() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(4); + assertThat(scan.planInputPartitions().length).isEqualTo(4); } private SparkScanBuilder scanBuilder() throws Exception { @@ -707,7 +725,7 @@ private SparkScanBuilder scanBuilder() throws Exception { } private void pushFilters(ScanBuilder scan, Predicate... predicates) { - Assertions.assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); + assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); SupportsPushDownV2Filters filterable = (SupportsPushDownV2Filters) scan; filterable.pushPredicates(predicates); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 1c08744f5662..7fe5951ef1be 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.expressions.Expressions.ref; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; import java.io.File; @@ -33,6 +34,7 @@ import org.apache.iceberg.DataOperations; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Files; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -45,7 +47,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; 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.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.spark.api.java.function.VoidFunction2; import org.apache.spark.sql.Dataset; @@ -55,21 +57,14 @@ import org.apache.spark.sql.streaming.DataStreamWriter; import org.apache.spark.sql.streaming.OutputMode; import org.apache.spark.sql.streaming.StreamingQuery; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public final class TestStructuredStreamingRead3 extends SparkCatalogTestBase { - public TestStructuredStreamingRead3( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public final class TestStructuredStreamingRead3 extends CatalogTestBase { private Table table; @@ -111,13 +106,13 @@ public TestStructuredStreamingRead3( Lists.newArrayList( new SimpleRecord(15, "fifteen"), new SimpleRecord(16, "sixteen")))); - @BeforeClass + @BeforeAll public static void setupSpark() { // disable AQE as tests assume that writes generate a particular number of files spark.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false"); } - @Before + @BeforeEach public void setupTable() { sql( "CREATE TABLE %s " @@ -129,19 +124,19 @@ public void setupTable() { microBatches.set(0); } - @After + @AfterEach public void stopStreams() throws TimeoutException { for (StreamingQuery query : spark.streams().active()) { query.stop(); } } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testReadStreamOnIcebergTableWithMultipleSnapshots() throws Exception { List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(expected); @@ -149,61 +144,63 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots() throws Exception StreamingQuery query = startStream(); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } - @Test + @TestTemplate public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfFiles_1() throws Exception { appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); - Assert.assertEquals( - 6, - microBatchCount( - ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1"))); + assertThat( + microBatchCount( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1"))) + .isEqualTo(6); } - @Test + @TestTemplate public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfFiles_2() throws Exception { appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); - Assert.assertEquals( - 3, - microBatchCount( - ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "2"))); + assertThat( + microBatchCount( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "2"))) + .isEqualTo(3); } - @Test + @TestTemplate public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfRows_1() throws Exception { appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); // only 1 micro-batch will be formed and we will read data partially - Assert.assertEquals( - 1, - microBatchCount(ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "1"))); + assertThat( + microBatchCount( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "1"))) + .isEqualTo(1); StreamingQuery query = startStream(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "1"); // check answer correctness only 1 record read the micro-batch will be stuck List actual = rowsAvailable(query); - Assertions.assertThat(actual) + assertThat(actual) .containsExactlyInAnyOrderElementsOf( Lists.newArrayList(TEST_DATA_MULTIPLE_SNAPSHOTS.get(0).get(0))); } - @Test + @TestTemplate public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfRows_4() throws Exception { appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); - Assert.assertEquals( - 2, - microBatchCount(ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "4"))); + assertThat( + microBatchCount( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "4"))) + .isEqualTo(2); } - @Test + @TestTemplate public void testReadStreamOnIcebergThenAddData() throws Exception { List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; @@ -212,10 +209,10 @@ public void testReadStreamOnIcebergThenAddData() throws Exception { appendDataAsMultipleSnapshots(expected); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } - @Test + @TestTemplate public void testReadingStreamFromTimestamp() throws Exception { List dataBeforeTimestamp = Lists.newArrayList( @@ -232,17 +229,17 @@ public void testReadingStreamFromTimestamp() throws Exception { startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(streamStartTimestamp)); List empty = rowsAvailable(query); - Assertions.assertThat(empty.isEmpty()).isTrue(); + assertThat(empty).isEmpty(); List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(expected); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } - @Test + @TestTemplate public void testReadingStreamFromFutureTimetsamp() throws Exception { long futureTimestamp = System.currentTimeMillis() + 10000; @@ -250,7 +247,7 @@ public void testReadingStreamFromFutureTimetsamp() throws Exception { startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(futureTimestamp)); List actual = rowsAvailable(query); - Assertions.assertThat(actual.isEmpty()).isTrue(); + assertThat(actual).isEmpty(); List data = Lists.newArrayList( @@ -263,7 +260,7 @@ public void testReadingStreamFromFutureTimetsamp() throws Exception { .forEach( x -> { appendData(data); - Assertions.assertThat(rowsAvailable(query).isEmpty()).isTrue(); + assertThat(rowsAvailable(query)).isEmpty(); }); waitUntilAfter(futureTimestamp); @@ -271,10 +268,10 @@ public void testReadingStreamFromFutureTimetsamp() throws Exception { // Data appended after the timestamp should appear appendData(data); actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(data); + assertThat(actual).containsExactlyInAnyOrderElementsOf(data); } - @Test + @TestTemplate public void testReadingStreamFromTimestampFutureWithExistingSnapshots() throws Exception { List dataBeforeTimestamp = Lists.newArrayList( @@ -287,17 +284,17 @@ public void testReadingStreamFromTimestampFutureWithExistingSnapshots() throws E StreamingQuery query = startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(streamStartTimestamp)); List actual = rowsAvailable(query); - Assert.assertEquals(Collections.emptyList(), actual); + assertThat(actual).isEmpty(); // Stream should contain data added after the timestamp elapses waitUntilAfter(streamStartTimestamp); List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(expected); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } - @Test + @TestTemplate public void testReadingStreamFromTimestampOfExistingSnapshot() throws Exception { List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; @@ -316,10 +313,10 @@ public void testReadingStreamFromTimestampOfExistingSnapshot() throws Exception } List actual = rowsAvailable(stream); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } - @Test + @TestTemplate public void testReadingStreamWithExpiredSnapshotFromTimestamp() throws TimeoutException { List firstSnapshotRecordList = Lists.newArrayList(new SimpleRecord(1, "one")); @@ -345,14 +342,14 @@ public void testReadingStreamWithExpiredSnapshotFromTimestamp() throws TimeoutEx startStream( SparkReadOptions.STREAM_FROM_TIMESTAMP, String.valueOf(firstSnapshotCommitTime)); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(expectedRecordList); + assertThat(actual).containsExactlyInAnyOrderElementsOf(expectedRecordList); } - @Test + @TestTemplate public void testResumingStreamReadFromCheckpoint() throws Exception { - File writerCheckpointFolder = temp.newFolder("writer-checkpoint-folder"); + File writerCheckpointFolder = temp.resolve("writer-checkpoint-folder").toFile(); File writerCheckpoint = new File(writerCheckpointFolder, "writer-checkpoint"); - File output = temp.newFolder(); + File output = temp.resolve("junit").toFile(); DataStreamWriter querySource = spark @@ -384,15 +381,15 @@ public void testResumingStreamReadFromCheckpoint() throws Exception { // Read data added by the stream List actual = spark.read().load(output.getPath()).as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } } - @Test + @TestTemplate public void testFailReadingCheckpointInvalidSnapshot() throws IOException, TimeoutException { - File writerCheckpointFolder = temp.newFolder("writer-checkpoint-folder"); + File writerCheckpointFolder = temp.resolve("writer-checkpoint-folder").toFile(); File writerCheckpoint = new File(writerCheckpointFolder, "writer-checkpoint"); - File output = temp.newFolder(); + File output = temp.resolve("junit").toFile(); DataStreamWriter querySource = spark @@ -428,7 +425,7 @@ public void testFailReadingCheckpointInvalidSnapshot() throws IOException, Timeo firstSnapshotid)); } - @Test + @TestTemplate public void testParquetOrcAvroDataInOneTable() throws Exception { List parquetFileRecords = Lists.newArrayList( @@ -445,19 +442,19 @@ public void testParquetOrcAvroDataInOneTable() throws Exception { appendData(avroFileRecords, "avro"); StreamingQuery query = startStream(); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf( Iterables.concat(parquetFileRecords, orcFileRecords, avroFileRecords)); } - @Test + @TestTemplate public void testReadStreamFromEmptyTable() throws Exception { StreamingQuery stream = startStream(); List actual = rowsAvailable(stream); - Assert.assertEquals(Collections.emptyList(), actual); + assertThat(actual).isEmpty(); } - @Test + @TestTemplate public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception { // upgrade table to version 2 - to facilitate creation of Snapshot of type OVERWRITE. TableOperations ops = ((BaseTable) table).operations(); @@ -478,7 +475,7 @@ public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception DeleteFile eqDeletes = FileHelpers.writeDeleteFile( table, - Files.localOutput(temp.newFile()), + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), dataDeletes, deleteRowSchema); @@ -487,17 +484,17 @@ public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception // check pre-condition - that the above Delete file write - actually resulted in snapshot of // type OVERWRITE - Assert.assertEquals(DataOperations.OVERWRITE, table.currentSnapshot().operation()); + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.OVERWRITE); StreamingQuery query = startStream(); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .cause() .isInstanceOf(IllegalStateException.class) .hasMessageStartingWith("Cannot process overwrite snapshot"); } - @Test + @TestTemplate public void testReadStreamWithSnapshotTypeReplaceIgnoresReplace() throws Exception { // fill table with some data List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; @@ -507,14 +504,14 @@ public void testReadStreamWithSnapshotTypeReplaceIgnoresReplace() throws Excepti table.rewriteManifests().clusterBy(f -> 1).commit(); // check pre-condition - Assert.assertEquals(DataOperations.REPLACE, table.currentSnapshot().operation()); + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.REPLACE); StreamingQuery query = startStream(); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } - @Test + @TestTemplate public void testReadStreamWithSnapshotTypeDeleteErrorsOut() throws Exception { table.updateSpec().removeField("id_bucket").addField(ref("id")).commit(); @@ -527,17 +524,17 @@ public void testReadStreamWithSnapshotTypeDeleteErrorsOut() throws Exception { // check pre-condition - that the above delete operation on table resulted in Snapshot of Type // DELETE. - Assert.assertEquals(DataOperations.DELETE, table.currentSnapshot().operation()); + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.DELETE); StreamingQuery query = startStream(); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .cause() .isInstanceOf(IllegalStateException.class) .hasMessageStartingWith("Cannot process delete snapshot"); } - @Test + @TestTemplate public void testReadStreamWithSnapshotTypeDeleteAndSkipDeleteOption() throws Exception { table.updateSpec().removeField("id_bucket").addField(ref("id")).commit(); @@ -550,14 +547,14 @@ public void testReadStreamWithSnapshotTypeDeleteAndSkipDeleteOption() throws Exc // check pre-condition - that the above delete operation on table resulted in Snapshot of Type // DELETE. - Assert.assertEquals(DataOperations.DELETE, table.currentSnapshot().operation()); + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.DELETE); StreamingQuery query = startStream(SparkReadOptions.STREAMING_SKIP_DELETE_SNAPSHOTS, "true"); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf(Iterables.concat(dataAcrossSnapshots)); } - @Test + @TestTemplate public void testReadStreamWithSnapshotTypeDeleteAndSkipOverwriteOption() throws Exception { table.updateSpec().removeField("id_bucket").addField(ref("id")).commit(); @@ -570,10 +567,10 @@ public void testReadStreamWithSnapshotTypeDeleteAndSkipOverwriteOption() throws // check pre-condition - that the above delete operation on table resulted in Snapshot of Type // OVERWRITE. - Assert.assertEquals(DataOperations.OVERWRITE, table.currentSnapshot().operation()); + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.OVERWRITE); StreamingQuery query = startStream(SparkReadOptions.STREAMING_SKIP_OVERWRITE_SNAPSHOTS, "true"); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf(Iterables.concat(dataAcrossSnapshots)); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java index ac674e2e62e8..306444b9f29f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java @@ -19,9 +19,11 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.Files.localOutput; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.time.LocalDateTime; import java.util.List; import java.util.UUID; @@ -31,6 +33,9 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -41,25 +46,22 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkReadOptions; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.data.GenericsHelpers; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestTimestampWithoutZone extends SparkTestBase { +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestTimestampWithoutZone extends TestBase { private static final Configuration CONF = new Configuration(); private static final HadoopTables TABLES = new HadoopTables(CONF); @@ -71,24 +73,27 @@ public class TestTimestampWithoutZone extends SparkTestBase { private static SparkSession spark = null; - @BeforeClass + @BeforeAll public static void startSpark() { TestTimestampWithoutZone.spark = SparkSession.builder().master("local[2]").getOrCreate(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestTimestampWithoutZone.spark; TestTimestampWithoutZone.spark = null; currentSpark.stop(); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - private final String format; - private final boolean vectorized; + @Parameter(index = 0) + private String format; - @Parameterized.Parameters(name = "format = {0}, vectorized = {1}") + @Parameter(index = 1) + private boolean vectorized; + + @Parameters(name = "format = {0}, vectorized = {1}") public static Object[][] parameters() { return new Object[][] { {"parquet", false}, @@ -97,21 +102,16 @@ public static Object[][] parameters() { }; } - public TestTimestampWithoutZone(String format, boolean vectorized) { - this.format = format; - this.vectorized = vectorized; - } - private File parent = null; private File unpartitioned = null; private List records = null; - @Before + @BeforeEach public void writeUnpartitionedTable() throws IOException { - this.parent = temp.newFolder("TestTimestampWithoutZone"); + this.parent = temp.resolve("TestTimestampWithoutZone").toFile(); this.unpartitioned = new File(parent, "unpartitioned"); File dataFolder = new File(unpartitioned, "data"); - Assert.assertTrue("Mkdir should succeed", dataFolder.mkdirs()); + assertThat(dataFolder.mkdirs()).as("Mkdir should succeed").isTrue(); Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), unpartitioned.toString()); Schema tableSchema = table.schema(); // use the table schema because ids are reassigned @@ -138,12 +138,12 @@ public void writeUnpartitionedTable() throws IOException { table.newAppend().appendFile(file).commit(); } - @Test + @TestTemplate public void testUnpartitionedTimestampWithoutZone() { assertEqualsSafe(SCHEMA.asStruct(), records, read(unpartitioned.toString(), vectorized)); } - @Test + @TestTemplate public void testUnpartitionedTimestampWithoutZoneProjection() { Schema projection = SCHEMA.select("id", "ts"); assertEqualsSafe( @@ -152,7 +152,7 @@ public void testUnpartitionedTimestampWithoutZoneProjection() { read(unpartitioned.toString(), vectorized, "id", "ts")); } - @Test + @TestTemplate public void testUnpartitionedTimestampWithoutZoneAppend() { spark .read() @@ -182,7 +182,7 @@ private static Record projectFlat(Schema projection, Record record) { public static void assertEqualsSafe( Types.StructType struct, List expected, List actual) { - Assert.assertEquals("Number of results should match expected", expected.size(), actual.size()); + assertThat(actual).as("Number of results should match expected").hasSameSizeAs(expected); for (int i = 0; i < expected.size(); i += 1) { GenericsHelpers.assertEqualsSafe(struct, expected.get(i), actual.get(i)); }