From 5be0372514dda9500b3b28f39171b025db782fe8 Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Tue, 19 Dec 2023 19:18:18 +0530 Subject: [PATCH 01/15] spark/source dir: update tests to AssertJ non-parameterized --- .../iceberg/spark/source/TestBaseReader.java | 71 ++-- .../spark/source/TestChangelogReader.java | 31 +- .../spark/source/TestDataFrameWriterV2.java | 28 +- .../spark/source/TestDataSourceOptions.java | 90 ++--- .../source/TestForwardCompatibility.java | 35 +- .../source/TestIcebergSourceHadoopTables.java | 6 +- .../source/TestIcebergSourceHiveTables.java | 8 +- .../source/TestIcebergSourceTablesBase.java | 276 +++++++------- .../spark/source/TestIcebergSpark.java | 136 ++++--- .../TestMetadataTableReadableMetrics.java | 21 +- .../spark/source/TestPathIdentifier.java | 36 +- .../spark/source/TestReadProjection.java | 346 +++++++++--------- .../spark/source/TestSparkAggregates.java | 15 +- .../TestSparkCatalogCacheExpiration.java | 30 +- .../spark/source/TestSparkDataFile.java | 23 +- .../spark/source/TestSparkReadMetrics.java | 108 +++--- .../spark/source/TestSparkReadProjection.java | 2 +- .../spark/source/TestSparkStagedScan.java | 21 +- .../iceberg/spark/source/TestSparkTable.java | 20 +- .../spark/source/TestStreamingOffset.java | 12 +- .../spark/source/TestStructuredStreaming.java | 59 +-- .../spark/source/TestWriteMetricsConfig.java | 102 +++--- 22 files changed, 750 insertions(+), 726 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java index 3d94966eb76c..6adf07c5cec5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java @@ -20,9 +20,11 @@ import static org.apache.iceberg.FileFormat.PARQUET; 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.util.Iterator; import java.util.List; import java.util.Map; @@ -47,14 +49,13 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestBaseReader { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; private Table table; @@ -129,15 +130,13 @@ public void testClosureOnDataExhaustion() throws IOException { int countRecords = 0; while (reader.next()) { countRecords += 1; - Assert.assertNotNull("Reader should return non-null value", reader.get()); + assertThat(reader.get()).as("Reader should return non-null value").isNotNull(); } - Assert.assertEquals( - "Reader returned incorrect number of records", totalTasks * recordPerTask, countRecords); + assertThat(totalTasks * recordPerTask).as("Reader returned incorrect number of records").isEqualTo(countRecords); tasks.forEach( t -> - Assert.assertTrue( - "All iterators should be closed after read exhausion", reader.isIteratorClosed(t))); + assertThat(reader.isIteratorClosed(t)).as("All iterators should be closed after read exhausion").isTrue()); } @Test @@ -145,28 +144,23 @@ public void testClosureDuringIteration() throws IOException { Integer totalTasks = 2; Integer recordPerTask = 1; List tasks = createFileScanTasks(totalTasks, recordPerTask); - Assert.assertEquals(2, tasks.size()); + assertThat(tasks).hasSize(2); FileScanTask firstTask = tasks.get(0); FileScanTask secondTask = tasks.get(1); ClosureTrackingReader reader = new ClosureTrackingReader(table, tasks); // Total of 2 elements - Assert.assertTrue(reader.next()); - Assert.assertFalse( - "First iter should not be closed on its last element", reader.isIteratorClosed(firstTask)); - - Assert.assertTrue(reader.next()); - Assert.assertTrue( - "First iter should be closed after moving to second iter", - reader.isIteratorClosed(firstTask)); - Assert.assertFalse( - "Second iter should not be closed on its last element", - reader.isIteratorClosed(secondTask)); - - Assert.assertFalse(reader.next()); - Assert.assertTrue(reader.isIteratorClosed(firstTask)); - Assert.assertTrue(reader.isIteratorClosed(secondTask)); + assertThat(reader.next()).isTrue(); + assertThat(reader.isIteratorClosed(firstTask)).as("First iter should not be closed on its last element").isFalse(); + + assertThat(reader.next()).isTrue(); + assertThat(reader.isIteratorClosed(firstTask)).as("First iter should be closed after moving to second iter").isTrue(); + assertThat(reader.isIteratorClosed(secondTask)).as("Second iter should not be closed on its last element").isFalse(); + + assertThat(reader.next()).isFalse(); + assertThat(reader.isIteratorClosed(firstTask)).isTrue(); + assertThat(reader.isIteratorClosed(secondTask)).isTrue(); } @Test @@ -181,8 +175,7 @@ public void testClosureWithoutAnyRead() throws IOException { tasks.forEach( t -> - Assert.assertFalse( - "Iterator should not be created eagerly for tasks", reader.hasIterator(t))); + assertThat(reader.hasIterator(t)).as("Iterator should not be created eagerly for tasks").isFalse()); } @Test @@ -195,8 +188,8 @@ public void testExplicitClosure() throws IOException { Integer halfDataSize = (totalTasks * recordPerTask) / 2; for (int i = 0; i < halfDataSize; i++) { - Assert.assertTrue("Reader should have some element", reader.next()); - Assert.assertNotNull("Reader should return non-null value", reader.get()); + assertThat(reader.next()).as("Reader should have some element").isTrue(); + assertThat(reader.get()).as("Reader should return non-null value").isNotNull(); } reader.close(); @@ -206,8 +199,7 @@ public void testExplicitClosure() throws IOException { tasks.forEach( t -> { if (reader.hasIterator(t)) { - Assert.assertTrue( - "Iterator should be closed after read exhausion", reader.isIteratorClosed(t)); + assertThat(reader.isIteratorClosed(t)).as("Iterator should be closed after read exhausion").isTrue(); } }); } @@ -222,20 +214,17 @@ public void testIdempotentExplicitClosure() throws IOException { // Total 100 elements, only 5 iterators have been created for (int i = 0; i < 45; i++) { - Assert.assertTrue("eader should have some element", reader.next()); - Assert.assertNotNull("Reader should return non-null value", reader.get()); + assertThat(reader.next()).as("Reader should have some element").isTrue(); + assertThat(reader.get()).as("Reader should return non-null value").isNotNull(); } for (int closeAttempt = 0; closeAttempt < 5; closeAttempt++) { reader.close(); for (int i = 0; i < 5; i++) { - Assert.assertTrue( - "Iterator should be closed after read exhausion", - reader.isIteratorClosed(tasks.get(i))); + assertThat(reader.isIteratorClosed(tasks.get(i))).as("Iterator should be closed after read exhausion").isTrue(); } for (int i = 5; i < 10; i++) { - Assert.assertFalse( - "Iterator should not be created eagerly for tasks", reader.hasIterator(tasks.get(i))); + assertThat(reader.hasIterator(tasks.get(i))).as("Iterator should not be created eagerly for tasks").isFalse(); } } } @@ -243,10 +232,10 @@ public void testIdempotentExplicitClosure() throws IOException { private List createFileScanTasks(Integer totalTasks, Integer recordPerTask) throws IOException { String desc = "make_scan_tasks"; - File parent = temp.newFolder(desc); + File parent = temp.resolve(desc).toFile(); 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(); Schema schema = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java index fc17547fad41..39f228bd68cb 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java @@ -20,8 +20,10 @@ 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.IOException; +import java.nio.file.Path; import java.util.List; import java.util.stream.Collectors; import org.apache.iceberg.ChangelogOperation; @@ -41,17 +43,15 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.InternalRow; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestChangelogReader extends SparkTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestChangelogReader extends TestBase { private static final Schema SCHEMA = new Schema( required(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); @@ -64,9 +64,10 @@ public class TestChangelogReader extends SparkTestBase { private DataFile dataFile1; private DataFile dataFile2; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; - @Before + @BeforeEach public void before() throws IOException { table = catalog.createTable(TableIdentifier.of("default", "test"), SCHEMA, SPEC); // create some data @@ -85,7 +86,7 @@ public void before() throws IOException { dataFile2 = writeDataFile(records2); } - @After + @AfterEach public void after() { catalog.dropTable(TableIdentifier.of("default", "test")); } @@ -176,7 +177,7 @@ public void testDataFileRewrite() throws IOException { reader.close(); } - Assert.assertEquals("Should have no rows", 0, rows.size()); + assertThat(rows).as("Should have no rows").hasSize(0); } @Test @@ -254,6 +255,8 @@ private Object[] toJava(InternalRow row) { private DataFile writeDataFile(List records) throws IOException { // records all use IDs that are in bucket id_bucket=0 return FileHelpers.writeDataFile( - table, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), records); + table, + Files.localOutput(temp.toFile()), + TestHelpers.Row.of(0), records); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java index 6fbad46f96db..b583acf443e3 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java @@ -17,12 +17,14 @@ * under the License. */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; @@ -32,19 +34,17 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class TestDataFrameWriterV2 extends SparkTestBaseWithCatalog { - @Before +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestDataFrameWriterV2 extends TestBaseWithCatalog { + @BeforeEach public void createTable() { sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } @@ -76,7 +76,7 @@ public void testMergeSchemaFailsWithoutWriterOption() throws Exception { // this has a different error message than the case without accept-any-schema because it uses // Iceberg checks - Assertions.assertThatThrownBy(() -> threeColDF.writeTo(tableName).append()) + assertThatThrownBy(() -> threeColDF.writeTo(tableName).append()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Field new_col not found in source schema"); } @@ -102,7 +102,7 @@ public void testMergeSchemaWithoutAcceptAnySchema() throws Exception { "{ \"id\": 3, \"data\": \"c\", \"new_col\": 12.06 }", "{ \"id\": 4, \"data\": \"d\", \"new_col\": 14.41 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> threeColDF.writeTo(tableName).option("merge-schema", "true").append()) .isInstanceOf(AnalysisException.class) .hasMessageContaining( @@ -201,12 +201,12 @@ public void testWriteWithCaseSensitiveOption() throws NoSuchTableException, Pars List fields = Spark3Util.loadIcebergTable(sparkSession, tableName).schema().asStruct().fields(); // Additional columns should not be created - Assert.assertEquals(2, fields.size()); + assertThat(fields).hasSize(2); // enable spark.sql.caseSensitive sparkSession.sql(String.format("SET %s=true", SQLConf.CASE_SENSITIVE().key())); ds.writeTo(tableName).option("merge-schema", "true").option("check-ordering", "false").append(); fields = Spark3Util.loadIcebergTable(sparkSession, tableName).schema().asStruct().fields(); - Assert.assertEquals(4, fields.size()); + assertThat(fields).hasSize(4); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 83d8953735c5..480319bf1f4e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -19,9 +19,12 @@ package org.apache.iceberg.spark.source; 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.math.RoundingMode; +import java.nio.file.Path; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -43,8 +46,8 @@ import org.apache.iceberg.relocated.com.google.common.math.LongMath; import org.apache.iceberg.spark.CommitMetadata; import org.apache.iceberg.spark.SparkReadOptions; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.Column; @@ -54,15 +57,12 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -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.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; -public class TestDataSourceOptions extends SparkTestBaseWithCatalog { +public class TestDataSourceOptions extends TestBaseWithCatalog { private static final Configuration CONF = new Configuration(); private static final Schema SCHEMA = @@ -70,14 +70,15 @@ public class TestDataSourceOptions extends SparkTestBaseWithCatalog { optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); private static SparkSession spark = null; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; - @BeforeClass + @BeforeAll public static void startSpark() { TestDataSourceOptions.spark = SparkSession.builder().master("local[2]").getOrCreate(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestDataSourceOptions.spark; TestDataSourceOptions.spark = null; @@ -86,7 +87,7 @@ public static void stopSpark() { @Test public void testWriteFormatOptionOverridesTableProperties() 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(); @@ -109,14 +110,14 @@ public void testWriteFormatOptionOverridesTableProperties() throws IOException { tasks.forEach( task -> { FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); - Assert.assertEquals(FileFormat.PARQUET, fileFormat); + assertThat(fileFormat).isEqualTo(FileFormat.PARQUET); }); } } @Test public void testNoWriteFormatOption() 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(); @@ -134,14 +135,14 @@ public void testNoWriteFormatOption() throws IOException { tasks.forEach( task -> { FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); - Assert.assertEquals(FileFormat.AVRO, fileFormat); + assertThat(fileFormat).isEqualTo(FileFormat.AVRO); }); } } @Test public void testHadoopOptions() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); Configuration sparkHadoopConf = spark.sessionState().newHadoopConf(); String originalDefaultFS = sparkHadoopConf.get("fs.default.name"); @@ -175,7 +176,7 @@ public void testHadoopOptions() throws IOException { List resultRecords = resultDf.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Records should match", expectedRecords, resultRecords); + assertThat(resultRecords).as("Records should match").isEqualTo(expectedRecords); } finally { sparkHadoopConf.set("fs.default.name", originalDefaultFS); } @@ -183,7 +184,7 @@ public void testHadoopOptions() throws IOException { @Test public void testSplitOptionsOverridesTableProperties() 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(); @@ -207,7 +208,7 @@ public void testSplitOptionsOverridesTableProperties() throws IOException { List files = Lists.newArrayList(icebergTable.currentSnapshot().addedDataFiles(icebergTable.io())); - Assert.assertEquals("Should have written 1 file", 1, files.size()); + assertThat(files).as("Should have written 1 file").hasSize(1); long fileSize = files.get(0).fileSizeInBytes(); long splitSize = LongMath.divide(fileSize, 2, RoundingMode.CEILING); @@ -219,12 +220,12 @@ public void testSplitOptionsOverridesTableProperties() throws IOException { .option(SparkReadOptions.SPLIT_SIZE, String.valueOf(splitSize)) .load(tableLocation); - Assert.assertEquals("Spark partitions should match", 2, resultDf.javaRDD().getNumPartitions()); + assertThat(2).as("Spark partitions should match").isEqualTo(resultDf.javaRDD().getNumPartitions()); } @Test public void testIncrementalScanOptions() 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(); @@ -245,7 +246,7 @@ public void testIncrementalScanOptions() throws IOException { List snapshotIds = SnapshotUtil.currentAncestorIds(table); // start-snapshot-id and snapshot-id are both configured. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -259,7 +260,7 @@ public void testIncrementalScanOptions() throws IOException { "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // end-snapshot-id and as-of-timestamp are both configured. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -275,7 +276,7 @@ public void testIncrementalScanOptions() throws IOException { "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // only end-snapshot-id is configured. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -297,7 +298,7 @@ public void testIncrementalScanOptions() throws IOException { .orderBy("id") .as(Encoders.bean(SimpleRecord.class)) .collectAsList(); - Assert.assertEquals("Records should match", expectedRecords.subList(1, 4), result); + assertThat(expectedRecords.subList(1, 4)).as("Records should match").isEqualTo(result); // test (2nd snapshot, 3rd snapshot] incremental scan. Dataset resultDf = @@ -309,13 +310,13 @@ public void testIncrementalScanOptions() throws IOException { .load(tableLocation); List result1 = resultDf.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Records should match", expectedRecords.subList(2, 3), result1); - Assert.assertEquals("Unprocessed count should match record count", 1, resultDf.count()); + assertThat(expectedRecords.subList(2, 3)).as("Records should match").isEqualTo(result1); + assertThat(1).as("Unprocessed count should match record count").isEqualTo(resultDf.count()); } @Test public void testMetadataSplitSizeOptionOverrideTableProperties() 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(); @@ -332,7 +333,7 @@ public void testMetadataSplitSizeOptionOverrideTableProperties() throws IOExcept List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Must be 2 manifests", 2, manifests.size()); + assertThat(manifests).as("Must be 2 manifests").hasSize(2); // set the target metadata split size so each manifest ends up in a separate split table @@ -341,7 +342,7 @@ public void testMetadataSplitSizeOptionOverrideTableProperties() throws IOExcept .commit(); Dataset entriesDf = spark.read().format("iceberg").load(tableLocation + "#entries"); - Assert.assertEquals("Num partitions must match", 2, entriesDf.javaRDD().getNumPartitions()); + assertThat(2).as("Num partitions must match").isEqualTo(entriesDf.javaRDD().getNumPartitions()); // override the table property using options entriesDf = @@ -350,12 +351,12 @@ public void testMetadataSplitSizeOptionOverrideTableProperties() throws IOExcept .format("iceberg") .option(SparkReadOptions.SPLIT_SIZE, String.valueOf(128 * 1024 * 1024)) .load(tableLocation + "#entries"); - Assert.assertEquals("Num partitions must match", 1, entriesDf.javaRDD().getNumPartitions()); + assertThat(1).as("Num partitions must match").isEqualTo(entriesDf.javaRDD().getNumPartitions()); } @Test public void testDefaultMetadataSplitSize() 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(); @@ -384,12 +385,12 @@ public void testDefaultMetadataSplitSize() throws IOException { Dataset metadataDf = spark.read().format("iceberg").load(tableLocation + "#entries"); int partitionNum = metadataDf.javaRDD().getNumPartitions(); - Assert.assertEquals("Spark partitions should match", expectedSplits, partitionNum); + assertThat(expectedSplits).as("Spark partitions should match").isEqualTo(partitionNum); } @Test public void testExtraSnapshotMetadata() throws IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); tables.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableLocation); @@ -407,13 +408,13 @@ public void testExtraSnapshotMetadata() throws IOException { Table table = tables.load(tableLocation); - Assert.assertTrue(table.currentSnapshot().summary().get("extra-key").equals("someValue")); - Assert.assertTrue(table.currentSnapshot().summary().get("another-key").equals("anotherValue")); + assertThat(table.currentSnapshot().summary().get("extra-key")).isEqualTo("someValue"); + assertThat(table.currentSnapshot().summary().get("another-key")).isEqualTo("anotherValue"); } @Test public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOException { - String tableLocation = temp.newFolder("iceberg-table").toString(); + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); Table table = @@ -448,9 +449,9 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx writerThread.join(); List snapshots = Lists.newArrayList(table.snapshots()); - Assert.assertEquals(2, snapshots.size()); - Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assertions.assertThat(snapshots.get(1).summary()) + assertThat(snapshots).hasSize(2); + assertThat(snapshots.get(0).summary().get("writer-thread")).isNull(); + assertThat(snapshots.get(1).summary()) .containsEntry("writer-thread", "test-extra-commit-message-writer-thread") .containsEntry("extra-key", "someValue") .containsEntry("another-key", "anotherValue"); @@ -491,9 +492,10 @@ public void testExtraSnapshotMetadataWithDelete() Table table = validationCatalog.loadTable(tableIdent); List snapshots = Lists.newArrayList(table.snapshots()); - Assert.assertEquals(2, snapshots.size()); - Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assertions.assertThat(snapshots.get(1).summary()) + + assertThat(snapshots).hasSize(2); + assertThat(snapshots.get(0).summary().get("writer-thread")).isNull(); + assertThat(snapshots.get(1).summary()) .containsEntry("writer-thread", "test-extra-commit-message-delete-thread") .containsEntry("extra-key", "someValue") .containsEntry("another-key", "anotherValue"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 446989d1af3e..5972ee4af1dd 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -20,9 +20,12 @@ import static org.apache.iceberg.Files.localInput; import static org.apache.iceberg.Files.localOutput; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.UUID; import java.util.concurrent.TimeoutException; @@ -54,13 +57,10 @@ import org.apache.spark.sql.execution.streaming.MemoryStream; import org.apache.spark.sql.streaming.StreamingQuery; import org.apache.spark.sql.streaming.StreamingQueryException; -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.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import scala.Option; import scala.collection.JavaConverters; @@ -87,16 +87,17 @@ public class TestForwardCompatibility { .addField("identity", 1, "id_zero") .build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; private static SparkSession spark = null; - @BeforeClass + @BeforeAll public static void startSpark() { TestForwardCompatibility.spark = SparkSession.builder().master("local[2]").getOrCreate(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestForwardCompatibility.spark; TestForwardCompatibility.spark = null; @@ -105,7 +106,7 @@ public static void stopSpark() { @Test public void testSparkWriteFailsUnknownTransform() throws IOException { - File parent = temp.newFolder("avro"); + File parent = temp.resolve("avro").toFile(); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); dataFolder.mkdirs(); @@ -119,7 +120,7 @@ public void testSparkWriteFailsUnknownTransform() throws IOException { Dataset df = spark.createDataFrame(expected, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> df.select("id", "data") .write() @@ -132,7 +133,7 @@ public void testSparkWriteFailsUnknownTransform() throws IOException { @Test public void testSparkStreamingWriteFailsUnknownTransform() throws IOException, TimeoutException { - File parent = temp.newFolder("avro"); + File parent = temp.resolve("avro").toFile(); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); dataFolder.mkdirs(); @@ -157,14 +158,14 @@ public void testSparkStreamingWriteFailsUnknownTransform() throws IOException, T List batch1 = Lists.newArrayList(1, 2); send(batch1, inputStream); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .isInstanceOf(StreamingQueryException.class) .hasMessageEndingWith("Cannot write using unsupported transforms: zero"); } @Test public void testSparkCanReadUnknownTransform() throws IOException { - File parent = temp.newFolder("avro"); + File parent = temp.resolve("avro").toFile(); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); dataFolder.mkdirs(); @@ -194,7 +195,7 @@ public void testSparkCanReadUnknownTransform() throws IOException { .withPartitionPath("id_zero=0") .build(); - OutputFile manifestFile = localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + OutputFile manifestFile = localOutput(FileFormat.AVRO.addExtension(temp.toFile().toString())); ManifestWriter manifestWriter = ManifestFiles.write(FAKE_SPEC, manifestFile); try { manifestWriter.add(file); @@ -207,7 +208,7 @@ public void testSparkCanReadUnknownTransform() throws IOException { Dataset df = spark.read().format("iceberg").load(location.toString()); List rows = df.collectAsList(); - Assert.assertEquals("Should contain 100 rows", 100, rows.size()); + assertThat(rows).as("Should contain 100 rows").hasSize(100); for (int i = 0; i < expected.size(); i += 1) { TestHelpers.assertEqualsSafe(table.schema().asStruct(), expected.get(i), rows.get(i)); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java index 746415818c84..ff0b76ed0e3f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java @@ -26,7 +26,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopTables; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; public class TestIcebergSourceHadoopTables extends TestIcebergSourceTablesBase { @@ -35,9 +35,9 @@ public class TestIcebergSourceHadoopTables extends TestIcebergSourceTablesBase { File tableDir = null; String tableLocation = null; - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); + this.tableDir = temp.toFile(); tableDir.delete(); // created by table create this.tableLocation = tableDir.toURI().toString(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java index 2a264b74b0e2..9120bbcc35a3 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java @@ -27,14 +27,14 @@ import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.After; -import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; public class TestIcebergSourceHiveTables extends TestIcebergSourceTablesBase { private static TableIdentifier currentIdentifier; - @BeforeClass + @BeforeAll public static void start() { Namespace db = Namespace.of("db"); if (!catalog.namespaceExists(db)) { @@ -42,7 +42,7 @@ public static void start() { } } - @After + @AfterEach public void dropTable() throws IOException { if (!catalog.tableExists(currentIdentifier)) { return; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 486713e52e30..003e5e0769ed 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -22,10 +22,13 @@ import static org.apache.iceberg.ManifestContent.DELETES; 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.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.file.Path; import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.Arrays; @@ -74,8 +77,8 @@ import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkTableUtil; -import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.actions.SparkActions; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.Types; @@ -91,14 +94,11 @@ import org.apache.spark.sql.functions; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; -public abstract class TestIcebergSourceTablesBase extends SparkTestBase { +public abstract class TestIcebergSourceTablesBase extends TestBase { private static final Schema SCHEMA = new Schema( @@ -117,7 +117,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase { private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("id").build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; public abstract Table createTable( TableIdentifier ident, Schema schema, PartitionSpec spec, Map properties); @@ -130,7 +131,7 @@ public abstract Table createTable( public abstract void dropTable(TableIdentifier ident) throws IOException; - @After + @AfterEach public void removeTable() { spark.sql("DROP TABLE IF EXISTS parquet_table"); } @@ -160,7 +161,7 @@ public synchronized void testTablesSupport() { List actualRecords = resultDf.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Records should match", expectedRecords, actualRecords); + assertThat(actualRecords).as("Records should match").isEqualTo(expectedRecords); } @Test @@ -187,8 +188,7 @@ public void testEntriesTable() throws Exception { Snapshot snapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should only contain one manifest", 1, snapshot.allManifests(table.io()).size()); + assertThat(snapshot.allManifests(table.io())).as("Should only contain one manifest").hasSize(1); InputFile manifest = table.io().newInputFile(snapshot.allManifests(table.io()).get(0).path()); List expected = Lists.newArrayList(); @@ -205,8 +205,8 @@ public void testEntriesTable() throws Exception { }); } - Assert.assertEquals("Entries table should have one row", 1, expected.size()); - Assert.assertEquals("Actual results should have one row", 1, actual.size()); + assertThat(expected).as("Entries table should have one row").hasSize(1); + assertThat(actual).as("Actual results should have one row").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(entriesTableDs), expected.get(0), actual.get(0)); } @@ -236,8 +236,8 @@ public void testEntriesTablePartitionedPrune() { .select("status") .collectAsList(); - Assert.assertEquals("Results should contain only one status", 1, actual.size()); - Assert.assertEquals("That status should be Added (1)", 1, actual.get(0).getInt(0)); + assertThat(actual).as("Results should contain only one status").hasSize(1); + assertThat(1).as("That status should be Added (1)").isEqualTo(actual.get(0).getInt(0)); } @Test @@ -408,8 +408,9 @@ public void testAllEntriesTable() throws Exception { expected.sort(Comparator.comparing(o -> (Long) o.get("snapshot_id"))); - Assert.assertEquals("Entries table should have 3 rows", 3, expected.size()); - Assert.assertEquals("Actual results should have 3 rows", 3, actual.size()); + assertThat(expected).as("Entries table should have 3 rows").hasSize(3); + assertThat(actual).as("Actual results should have 3 rows").hasSize(3); + for (int i = 0; i < expected.size(); i += 1) { TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(entriesTableDs), expected.get(i), actual.get(i)); @@ -434,16 +435,14 @@ public void testCountEntriesTable() { final int expectedEntryCount = 1; // count entries - Assert.assertEquals( - "Count should return " + expectedEntryCount, - expectedEntryCount, - spark.read().format("iceberg").load(loadLocation(tableIdentifier, "entries")).count()); + assertThat(expectedEntryCount) + .as("Count should return " + expectedEntryCount) + .isEqualTo(spark.read().format("iceberg").load(loadLocation(tableIdentifier, "entries")).count()); // count all_entries - Assert.assertEquals( - "Count should return " + expectedEntryCount, - expectedEntryCount, - spark.read().format("iceberg").load(loadLocation(tableIdentifier, "all_entries")).count()); + assertThat(expectedEntryCount) + .as("Count should return " + expectedEntryCount) + .isEqualTo(spark.read().format("iceberg").load(loadLocation(tableIdentifier, "all_entries")).count()); } @Test @@ -492,8 +491,8 @@ public void testFilesTable() throws Exception { } } - Assert.assertEquals("Files table should have one row", 1, expected.size()); - Assert.assertEquals("Actual results should have one row", 1, actual.size()); + assertThat(expected).as("Files table should have one row").hasSize(1); + assertThat(actual).as("Actual results should have one row").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(filesTableDs), expected.get(0), actual.get(0)); @@ -510,7 +509,7 @@ public void testFilesTableWithSnapshotIdInheritance() throws Exception { String.format( "CREATE TABLE parquet_table (data string, id int) " + "USING parquet PARTITIONED BY (id) LOCATION '%s'", - temp.newFolder())); + temp.toFile())); List records = Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); @@ -548,8 +547,8 @@ public void testFilesTableWithSnapshotIdInheritance() throws Exception { } Types.StructType struct = TestHelpers.nonDerivedSchema(filesTableDs); - Assert.assertEquals("Files table should have one row", 2, expected.size()); - Assert.assertEquals("Actual results should have one row", 2, actual.size()); + assertThat(expected).as("Files table should have 2 rows").hasSize(2); + assertThat(actual).as("Actual results should have 2 rows").hasSize(2); TestHelpers.assertEqualsSafe(struct, expected.get(0), actual.get(0)); TestHelpers.assertEqualsSafe(struct, expected.get(1), actual.get(1)); } @@ -566,7 +565,7 @@ public void testV1EntriesTableWithSnapshotIdInheritance() throws Exception { String.format( "CREATE TABLE parquet_table (data string, id int) " + "USING parquet PARTITIONED BY (id) LOCATION '%s'", - temp.newFolder())); + temp.toFile())); List records = Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); @@ -593,11 +592,12 @@ public void testV1EntriesTableWithSnapshotIdInheritance() throws Exception { long snapshotId = table.currentSnapshot().snapshotId(); - Assert.assertEquals("Entries table should have 2 rows", 2, actual.size()); - Assert.assertEquals("Sequence number must match", 0, actual.get(0).getLong(0)); - Assert.assertEquals("Snapshot id must match", snapshotId, actual.get(0).getLong(1)); - Assert.assertEquals("Sequence number must match", 0, actual.get(1).getLong(0)); - Assert.assertEquals("Snapshot id must match", snapshotId, actual.get(1).getLong(1)); + + assertThat(actual).as("Entries table should have 2 rows").hasSize(2); + assertThat(0).as("Sequence number must match").isEqualTo(actual.get(0).getLong(0)); + assertThat(snapshotId).as("Snapshot id must match").isEqualTo(actual.get(0).getLong(1)); + assertThat(0).as("Sequence number must match").isEqualTo(actual.get(1).getLong(0)); + assertThat(snapshotId).as("Snapshot id must match").isEqualTo(actual.get(1).getLong(1)); } @Test @@ -650,8 +650,8 @@ public void testFilesUnpartitionedTable() throws Exception { } } - Assert.assertEquals("Files table should have one row", 1, expected.size()); - Assert.assertEquals("Actual results should have one row", 1, actual.size()); + assertThat(expected).as("Files table should have one row").hasSize(1); + assertThat(actual).as("Actual results should have one row").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(filesTableDs), expected.get(0), actual.get(0)); } @@ -702,12 +702,11 @@ public void testAllMetadataTablesWithStagedCommits() { .load(loadLocation(tableIdentifier, "all_entries")) .collectAsList(); - Assert.assertTrue( - "Stage table should have some snapshots", table.snapshots().iterator().hasNext()); - Assert.assertNull("Stage table should have null currentSnapshot", table.currentSnapshot()); - Assert.assertEquals("Actual results should have two rows", 2, actualAllData.size()); - Assert.assertEquals("Actual results should have two rows", 2, actualAllManifests.size()); - Assert.assertEquals("Actual results should have two rows", 2, actualAllEntries.size()); + assertThat(table.snapshots().iterator().hasNext()).as("Stage table should have some snapshots").isTrue(); + assertThat(table.currentSnapshot()).as("Stage table should have null currentSnapshot").isNull(); + assertThat(actualAllData).as("Actual results should have two rows").hasSize(2); + assertThat(actualAllManifests).as("Actual results should have two rows").hasSize(2); + assertThat(actualAllEntries).as("Actual results should have two rows").hasSize(2); } @Test @@ -765,8 +764,8 @@ public void testAllDataFilesTable() throws Exception { expected.sort(Comparator.comparing(o -> o.get("file_path").toString())); - Assert.assertEquals("Files table should have two rows", 2, expected.size()); - Assert.assertEquals("Actual results should have two rows", 2, actual.size()); + assertThat(expected).as("Files table should have two rows").hasSize(2); + assertThat(actual).as("Actual results should have two rows").hasSize(2); for (int i = 0; i < expected.size(); i += 1) { TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(filesTableDs), expected.get(i), actual.get(i)); @@ -857,7 +856,7 @@ public void testHistoryTable() { .set("is_current_ancestor", true) .build()); - Assert.assertEquals("History table should have a row for each commit", 4, actual.size()); + assertThat(actual).as("History table should have a row for each commit").hasSize(4); TestHelpers.assertEqualsSafe(historyTable.schema().asStruct(), expected.get(0), actual.get(0)); TestHelpers.assertEqualsSafe(historyTable.schema().asStruct(), expected.get(1), actual.get(1)); TestHelpers.assertEqualsSafe(historyTable.schema().asStruct(), expected.get(2), actual.get(2)); @@ -936,7 +935,7 @@ public void testSnapshotsTable() { "total-data-files", "0")) .build()); - Assert.assertEquals("Snapshots table should have a row for each snapshot", 2, actual.size()); + assertThat(actual).as("Snapshots table should have a row for each snapshot").hasSize(2); TestHelpers.assertEqualsSafe(snapTable.schema().asStruct(), expected.get(0), actual.get(0)); TestHelpers.assertEqualsSafe(snapTable.schema().asStruct(), expected.get(1), actual.get(1)); } @@ -1009,7 +1008,7 @@ public void testPrunedSnapshotsTable() { "total-data-files", "0")) .build()); - Assert.assertEquals("Snapshots table should have a row for each snapshot", 2, actual.size()); + assertThat(actual).as("Snapshots table should have a row for each snapshot").hasSize(2); TestHelpers.assertEqualsSafe(projectedSchema.asStruct(), expected.get(0), actual.get(0)); TestHelpers.assertEqualsSafe(projectedSchema.asStruct(), expected.get(1), actual.get(1)); } @@ -1094,7 +1093,7 @@ public void testManifestsTable() { .build())) .build()); - Assert.assertEquals("Manifests table should have two manifest rows", 2, actual.size()); + assertThat(actual).as("Manifests table should have two manifest rows").hasSize(2); TestHelpers.assertEqualsSafe(manifestTable.schema().asStruct(), expected.get(0), actual.get(0)); TestHelpers.assertEqualsSafe(manifestTable.schema().asStruct(), expected.get(1), actual.get(1)); } @@ -1117,7 +1116,7 @@ public void testPruneManifestsTable() { if (!spark.version().startsWith("2")) { // Spark 2 isn't able to actually push down nested struct projections so this will not break - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -1175,7 +1174,7 @@ public void testPruneManifestsTable() { .build())) .build()); - Assert.assertEquals("Manifests table should have one manifest row", 1, actual.size()); + assertThat(actual).as("Manifests table should have one manifest row").hasSize(1); TestHelpers.assertEqualsSafe(projectedSchema.asStruct(), expected.get(0), actual.get(0)); } @@ -1227,7 +1226,7 @@ public void testAllManifestsTable() { .sorted(Comparator.comparing(o -> o.get("path").toString())) .collect(Collectors.toList()); - Assert.assertEquals("Manifests table should have 5 manifest rows", 5, actual.size()); + assertThat(actual).as("Manifests table should have 5 manifest rows").hasSize(5); for (int i = 0; i < expected.size(); i += 1) { TestHelpers.assertEqualsSafe( manifestTable.schema().asStruct(), expected.get(i), actual.get(i)); @@ -1290,10 +1289,9 @@ public void testUnpartitionedPartitionsTable() { Table partitionsTable = loadTable(tableIdentifier, "partitions"); - Assert.assertEquals( - "Schema should not have partition field", - expectedSchema, - partitionsTable.schema().asStruct()); + assertThat(expectedSchema) + .as("Schema should not have partition field") + .isEqualTo(partitionsTable.schema().asStruct()); GenericRecordBuilder builder = new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); @@ -1319,7 +1317,7 @@ public void testUnpartitionedPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .collectAsList(); - Assert.assertEquals("Unpartitioned partitions table should have one row", 1, actual.size()); + assertThat(actual).as("Unpartitioned partitions table should have one row").hasSize(1); TestHelpers.assertEqualsSafe(expectedSchema, expectedRow, actual.get(0)); } @@ -1400,8 +1398,8 @@ public void testPartitionsTable() { .set("last_updated_snapshot_id", secondCommitId) .build()); - Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); - Assert.assertEquals("Actual results should have two rows", 2, actual.size()); + assertThat(expected).as("Partitions table should have two rows").hasSize(2); + assertThat(actual).as("Actual results should have two rows").hasSize(2); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); @@ -1417,7 +1415,7 @@ public void testPartitionsTable() { .orderBy("partition.id") .collectAsList(); - Assert.assertEquals("Actual results should have one row", 1, actualAfterFirstCommit.size()); + assertThat(actualAfterFirstCommit).as("Actual results should have one row").hasSize(1); TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(0), actualAfterFirstCommit.get(0)); @@ -1429,7 +1427,8 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2") .collectAsList(); - Assert.assertEquals("Actual results should have one row", 1, filtered.size()); + + assertThat(filtered).as("Actual results should have one row").hasSize(1); TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); @@ -1440,7 +1439,8 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2 or record_count=1") .collectAsList(); - Assert.assertEquals("Actual results should have two row", 2, nonFiltered.size()); + + assertThat(nonFiltered).as("Actual results should have two rows").hasSize(2); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); @@ -1481,12 +1481,11 @@ public void testPartitionsTableLastUpdatedSnapshot() { // check if rewrite manifest does not override metadata about data file's creating snapshot RewriteManifests.Result rewriteManifestResult = SparkActions.get().rewriteManifests(table).execute(); - Assert.assertEquals( - "rewrite replaced 2 manifests", - 2, - Iterables.size(rewriteManifestResult.rewrittenManifests())); - Assert.assertEquals( - "rewrite added 1 manifests", 1, Iterables.size(rewriteManifestResult.addedManifests())); + assertThat(rewriteManifestResult.rewrittenManifests()) + .as("rewrite replaced 2 manifests").hasSize(2); + + assertThat(rewriteManifestResult.addedManifests()) + .as("rewrite added 1 manifests").hasSize(1); List actual = spark @@ -1538,8 +1537,8 @@ public void testPartitionsTableLastUpdatedSnapshot() { .set("last_updated_snapshot_id", secondCommitId) .build()); - Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); - Assert.assertEquals("Actual results should have two rows", 2, actual.size()); + assertThat(expected).as("Partitions table should have two rows").hasSize(2); + assertThat(actual).as("Actual results should have two rows").hasSize(2); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); @@ -1553,7 +1552,7 @@ public void testPartitionsTableLastUpdatedSnapshot() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2") .collectAsList(); - Assert.assertEquals("Actual results should have one row", 1, filtered.size()); + assertThat(filtered).as("Actual results should have one row").hasSize(2); TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); @@ -1584,8 +1583,7 @@ public void testPartitionsTableLastUpdatedSnapshot() { .format("iceberg") .load(loadLocation(tableIdentifier, "partitions")) .collectAsList(); - Assert.assertEquals( - "Actual results should have two row", 2, actualAfterSnapshotExpiration.size()); + assertThat(actualAfterSnapshotExpiration).as("Actual results should have two rows").hasSize(2); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), @@ -1634,7 +1632,7 @@ public void testPartitionsTableDeleteStats() { .load(loadLocation(tableIdentifier, "partitions")) .orderBy("partition.id") .collectAsList(); - Assert.assertEquals("Actual results should have two rows", 2, actual.size()); + assertThat(actual).as("Actual results should have two rows").hasSize(2); GenericRecordBuilder builder = new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); @@ -1693,7 +1691,7 @@ public void testPartitionsTableDeleteStats() { .load(loadLocation(tableIdentifier, "partitions")) .orderBy("partition.id") .collectAsList(); - Assert.assertEquals("Actual results should have two rows", 2, actual.size()); + assertThat(actual).as("Actual results should have two rows").hasSize(2); expected.remove(0); expected.add( 0, @@ -1735,8 +1733,9 @@ public synchronized void testSnapshotReadAfterAddColumn() { table.refresh(); Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", originalRecords, resultDf.orderBy("id").collectAsList()); + assertThat(originalRecords) + .as("Records should match") + .isEqualTo(resultDf.orderBy("id").collectAsList()); Snapshot snapshotBeforeAddColumn = table.currentSnapshot(); @@ -1765,8 +1764,9 @@ public synchronized void testSnapshotReadAfterAddColumn() { RowFactory.create(5, "xyz", "C")); Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", updatedRecords, resultDf2.orderBy("id").collectAsList()); + assertThat(updatedRecords) + .as("Records should match") + .isEqualTo(resultDf2.orderBy("id").collectAsList()); Dataset resultDf3 = spark @@ -1774,9 +1774,14 @@ public synchronized void testSnapshotReadAfterAddColumn() { .format("iceberg") .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", originalRecords, resultDf3.orderBy("id").collectAsList()); - Assert.assertEquals("Schemas should match", originalSparkSchema, resultDf3.schema()); + + assertThat(originalRecords) + .as("Records should match") + .isEqualTo(resultDf3.orderBy("id").collectAsList()); + + assertThat(originalSparkSchema) + .as("Schemas should match") + .isEqualTo(resultDf3.schema()); } @Test @@ -1802,8 +1807,10 @@ public synchronized void testSnapshotReadAfterDropColumn() { table.refresh(); Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", originalRecords, resultDf.orderBy("id").collectAsList()); + + assertThat(originalRecords) + .as("Records should match") + .isEqualTo(resultDf.orderBy("id").collectAsList()); long tsBeforeDropColumn = waitUntilAfter(System.currentTimeMillis()); table.updateSchema().deleteColumn("data").commit(); @@ -1831,8 +1838,9 @@ public synchronized void testSnapshotReadAfterDropColumn() { RowFactory.create(5, "C")); Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", updatedRecords, resultDf2.orderBy("id").collectAsList()); + assertThat(updatedRecords) + .as("Records should match") + .isEqualTo(resultDf2.orderBy("id").collectAsList()); Dataset resultDf3 = spark @@ -1840,9 +1848,14 @@ public synchronized void testSnapshotReadAfterDropColumn() { .format("iceberg") .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) .load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", originalRecords, resultDf3.orderBy("id").collectAsList()); - Assert.assertEquals("Schemas should match", originalSparkSchema, resultDf3.schema()); + + assertThat(originalRecords) + .as("Records should match") + .isEqualTo(resultDf3.orderBy("id").collectAsList()); + + assertThat(originalSparkSchema) + .as("Schemas should match") + .isEqualTo(resultDf3.schema()); // At tsAfterDropColumn, there has been a schema change, but no new snapshot, // so the snapshot as of tsAfterDropColumn is the same as that as of tsBeforeDropColumn. @@ -1852,9 +1865,14 @@ public synchronized void testSnapshotReadAfterDropColumn() { .format("iceberg") .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) .load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", originalRecords, resultDf4.orderBy("id").collectAsList()); - Assert.assertEquals("Schemas should match", originalSparkSchema, resultDf4.schema()); + + assertThat(originalRecords) + .as("Records should match") + .isEqualTo(resultDf4.orderBy("id").collectAsList()); + + assertThat(originalSparkSchema) + .as("Schemas should match") + .isEqualTo(resultDf4.schema()); } @Test @@ -1878,8 +1896,10 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { table.refresh(); Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", originalRecords, resultDf.orderBy("id").collectAsList()); + + assertThat(originalRecords) + .as("Records should match") + .isEqualTo(resultDf.orderBy("id").collectAsList()); Snapshot snapshotBeforeAddColumn = table.currentSnapshot(); @@ -1908,8 +1928,10 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { RowFactory.create(5, "xyz", "C")); Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", updatedRecords, resultDf2.orderBy("id").collectAsList()); + + assertThat(updatedRecords) + .as("Records should match") + .isEqualTo(resultDf2.orderBy("id").collectAsList()); table.updateSchema().deleteColumn("data").commit(); @@ -1922,8 +1944,10 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { RowFactory.create(5, "C")); Dataset resultDf3 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", recordsAfterDropColumn, resultDf3.orderBy("id").collectAsList()); + + assertThat(recordsAfterDropColumn) + .as("Records should match") + .isEqualTo(resultDf3.orderBy("id").collectAsList()); Dataset resultDf4 = spark @@ -1931,9 +1955,14 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { .format("iceberg") .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); - Assert.assertEquals( - "Records should match", originalRecords, resultDf4.orderBy("id").collectAsList()); - Assert.assertEquals("Schemas should match", originalSparkSchema, resultDf4.schema()); + + assertThat(originalRecords) + .as("Records should match") + .isEqualTo(resultDf4.orderBy("id").collectAsList()); + + assertThat(originalSparkSchema) + .as("Schemas should match") + .isEqualTo(resultDf4.schema()); } @Test @@ -1964,19 +1993,22 @@ public void testRemoveOrphanFilesActionSupport() throws InterruptedException { .location(table.location() + "/metadata") .olderThan(System.currentTimeMillis()) .execute(); - Assert.assertTrue( - "Should not delete any metadata files", Iterables.isEmpty(result1.orphanFileLocations())); + + assertThat(Iterables.isEmpty(result1.orphanFileLocations())) + .as("Should not delete any metadata files") + .isTrue(); DeleteOrphanFiles.Result result2 = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - Assert.assertEquals( - "Should delete 1 data file", 1, Iterables.size(result2.orphanFileLocations())); + + assertThat(result2.orphanFileLocations()) + .as("Should delete 1 data file").hasSize(1); Dataset resultDF = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); List actualRecords = resultDF.as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", records, actualRecords); + assertThat(records).as("Rows must match").isEqualTo(actualRecords); } @Test @@ -2016,7 +2048,7 @@ public void testFilesTablePartitionId() { .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) .collect(Collectors.toList()); - Assert.assertEquals("Should have two partition specs", ImmutableList.of(spec0, spec1), actual); + assertThat(ImmutableList.of(spec0, spec1)).as("Should have two partition specs").isEqualTo(actual); } @Test @@ -2050,7 +2082,7 @@ public void testAllManifestTableSnapshotFiltering() { table.refresh(); Snapshot snapshot2 = table.currentSnapshot(); - Assert.assertEquals("Should have two manifests", 2, snapshot2.allManifests(table.io()).size()); + assertThat(snapshot2.allManifests(table.io())).as("Should have two manifests").hasSize(2); snapshotIdToManifests.addAll( snapshot2.allManifests(table.io()).stream() .map(manifest -> Pair.of(snapshot2.snapshotId(), manifest)) @@ -2092,7 +2124,7 @@ public void testAllManifestTableSnapshotFiltering() { .sorted(Comparator.comparing(o -> o.get("path").toString())) .collect(Collectors.toList()); - Assert.assertEquals("Manifests table should have 3 manifest rows", 3, actual.size()); + assertThat(actual).as("Manifests table should have 3 manifest rows").hasSize(3); for (int i = 0; i < expected.size(); i += 1) { TestHelpers.assertEqualsSafe( manifestTable.schema().asStruct(), expected.get(i), actual.get(i)); @@ -2101,7 +2133,7 @@ public void testAllManifestTableSnapshotFiltering() { @Test public void testTableWithInt96Timestamp() throws IOException { - File parquetTableDir = temp.newFolder("table_timestamp_int96"); + File parquetTableDir = temp.resolve("table_timestamp_int96").toFile(); String parquetTableLocation = parquetTableDir.toURI().toString(); Schema schema = new Schema( @@ -2154,7 +2186,7 @@ public void testTableWithInt96Timestamp() throws IOException { .load(loadLocation(tableIdentifier)) .select("tmp_col") .collectAsList(); - Assertions.assertThat(actual) + assertThat(actual) .as("Rows must match") .containsExactlyInAnyOrderElementsOf(expected); dropTable(tableIdentifier); @@ -2256,7 +2288,7 @@ private DeleteFile writeEqDeleteFile(Table table) { try { return FileHelpers.writeDeleteFile( table, - Files.localOutput(temp.newFile()), + Files.localOutput(temp.toFile()), org.apache.iceberg.TestHelpers.Row.of(1), deletes, deleteRowSchema); @@ -2271,16 +2303,14 @@ private long totalSizeInBytes(Iterable dataFiles) { private void assertDataFilePartitions( List dataFiles, List expectedPartitionIds) { - Assert.assertEquals( - "Table should have " + expectedPartitionIds.size() + " data files", - expectedPartitionIds.size(), - dataFiles.size()); + assertThat(expectedPartitionIds) + .as("Table should have " + expectedPartitionIds.size() + " data files") + .hasSameSizeAs(dataFiles); for (int i = 0; i < dataFiles.size(); ++i) { - Assert.assertEquals( - "Data file should have partition of id " + expectedPartitionIds.get(i), - expectedPartitionIds.get(i).intValue(), - dataFiles.get(i).partition().get(0, Integer.class).intValue()); + assertThat(expectedPartitionIds.get(i).intValue()) + .as("Data file should have partition of id " + expectedPartitionIds.get(i)) + .isEqualTo(dataFiles.get(i).partition().get(0, Integer.class).intValue()); } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java index 37e329a8b97b..9bf9b03e8f08 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java @@ -17,6 +17,8 @@ * under the License. */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -33,22 +35,20 @@ import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DecimalType; import org.apache.spark.sql.types.VarcharType; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; public class TestIcebergSpark { private static SparkSession spark = null; - @BeforeClass + @BeforeAll public static void startSpark() { TestIcebergSpark.spark = SparkSession.builder().master("local[2]").getOrCreate(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestIcebergSpark.spark; TestIcebergSpark.spark = null; @@ -59,69 +59,64 @@ public static void stopSpark() { public void testRegisterIntegerBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_int_16", DataTypes.IntegerType, 16); List results = spark.sql("SELECT iceberg_bucket_int_16(1)").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1), - results.get(0).getInt(0)); + + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterShortBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_short_16", DataTypes.ShortType, 16); List results = spark.sql("SELECT iceberg_bucket_short_16(1S)").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1), - results.get(0).getInt(0)); + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterByteBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_byte_16", DataTypes.ByteType, 16); List results = spark.sql("SELECT iceberg_bucket_byte_16(1Y)").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1), - results.get(0).getInt(0)); + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterLongBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_long_16", DataTypes.LongType, 16); List results = spark.sql("SELECT iceberg_bucket_long_16(1L)").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) Transforms.bucket(16).bind(Types.LongType.get()).apply(1L), results.get(0).getInt(0)); + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16).bind(Types.LongType.get()).apply(1L)) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterStringBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_string_16", DataTypes.StringType, 16); List results = spark.sql("SELECT iceberg_bucket_string_16('hello')").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello"), - results.get(0).getInt(0)); + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterCharBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_char_16", new CharType(5), 16); List results = spark.sql("SELECT iceberg_bucket_char_16('hello')").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello"), - results.get(0).getInt(0)); + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterVarCharBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_varchar_16", new VarcharType(5), 16); List results = spark.sql("SELECT iceberg_bucket_varchar_16('hello')").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello"), - results.get(0).getInt(0)); + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")) + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -129,13 +124,12 @@ public void testRegisterDateBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_date_16", DataTypes.DateType, 16); List results = spark.sql("SELECT iceberg_bucket_date_16(DATE '2021-06-30')").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16) - .bind(Types.DateType.get()) - .apply(DateTimeUtils.fromJavaDate(Date.valueOf("2021-06-30"))), - results.get(0).getInt(0)); + .bind(Types.DateType.get()) + .apply(DateTimeUtils.fromJavaDate(Date.valueOf("2021-06-30")))) + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -146,42 +140,39 @@ public void testRegisterTimestampBucketUDF() { spark .sql("SELECT iceberg_bucket_timestamp_16(TIMESTAMP '2021-06-30 00:00:00.000')") .collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16) - .bind(Types.TimestampType.withZone()) - .apply( - DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2021-06-30 00:00:00.000"))), - results.get(0).getInt(0)); + .bind(Types.TimestampType.withZone()) + .apply( + DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2021-06-30 00:00:00.000")))) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterBinaryBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_binary_16", DataTypes.BinaryType, 16); List results = spark.sql("SELECT iceberg_bucket_binary_16(X'0020001F')").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16) - .bind(Types.BinaryType.get()) - .apply(ByteBuffer.wrap(new byte[] {0x00, 0x20, 0x00, 0x1F})), - results.get(0).getInt(0)); + .bind(Types.BinaryType.get()) + .apply(ByteBuffer.wrap(new byte[] {0x00, 0x20, 0x00, 0x1F}))) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterDecimalBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_decimal_16", new DecimalType(4, 2), 16); List results = spark.sql("SELECT iceberg_bucket_decimal_16(11.11)").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - (int) Transforms.bucket(16).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11")), - results.get(0).getInt(0)); + assertThat(results).hasSize(1); + assertThat((int) Transforms.bucket(16).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11"))) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterBooleanBucketUDF() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> IcebergSpark.registerBucketUDF( spark, "iceberg_bucket_boolean_16", DataTypes.BooleanType, 16)) @@ -191,7 +182,7 @@ public void testRegisterBooleanBucketUDF() { @Test public void testRegisterDoubleBucketUDF() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> IcebergSpark.registerBucketUDF( spark, "iceberg_bucket_double_16", DataTypes.DoubleType, 16)) @@ -201,7 +192,7 @@ public void testRegisterDoubleBucketUDF() { @Test public void testRegisterFloatBucketUDF() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> IcebergSpark.registerBucketUDF( spark, "iceberg_bucket_float_16", DataTypes.FloatType, 16)) @@ -213,37 +204,34 @@ public void testRegisterFloatBucketUDF() { public void testRegisterIntegerTruncateUDF() { IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_int_4", DataTypes.IntegerType, 4); List results = spark.sql("SELECT iceberg_truncate_int_4(1)").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - Transforms.truncate(4).bind(Types.IntegerType.get()).apply(1), results.get(0).getInt(0)); + assertThat(results).hasSize(1); + assertThat(Transforms.truncate(4).bind(Types.IntegerType.get()).apply(1)) + .isEqualTo(results.get(0).getInt(0)); } @Test public void testRegisterLongTruncateUDF() { IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_long_4", DataTypes.LongType, 4); List results = spark.sql("SELECT iceberg_truncate_long_4(1L)").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - Transforms.truncate(4).bind(Types.LongType.get()).apply(1L), results.get(0).getLong(0)); + assertThat(results).hasSize(1); + assertThat(Transforms.truncate(4).bind(Types.LongType.get()).apply(1L)) + .isEqualTo(results.get(0).getLong(0)); } @Test public void testRegisterDecimalTruncateUDF() { IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_decimal_4", new DecimalType(4, 2), 4); List results = spark.sql("SELECT iceberg_truncate_decimal_4(11.11)").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - Transforms.truncate(4).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11")), - results.get(0).getDecimal(0)); + assertThat(results).hasSize(1); + assertThat(Transforms.truncate(4).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11"))).isEqualTo(results.get(0).getDecimal(0)); } @Test public void testRegisterStringTruncateUDF() { IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_string_4", DataTypes.StringType, 4); List results = spark.sql("SELECT iceberg_truncate_string_4('hello')").collectAsList(); - Assert.assertEquals(1, results.size()); - Assert.assertEquals( - Transforms.truncate(4).bind(Types.StringType.get()).apply("hello"), - results.get(0).getString(0)); + assertThat(results).hasSize(1); + assertThat(Transforms.truncate(4).bind(Types.StringType.get()).apply("hello")) + .isEqualTo(results.get(0).getString(0)); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java index 9075257fa9f1..cb6f20f58085 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.util.Base64; import java.util.List; import java.util.Map; @@ -41,19 +42,19 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkCatalogConfig; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.junit.After; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; -public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog { +public class TestMetadataTableReadableMetrics extends TestBaseWithCatalog { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; private static final Types.StructType LEAF_STRUCT_TYPE = Types.StructType.of( @@ -125,7 +126,7 @@ private Table createPrimitiveTable() throws IOException { false, 2, 2L, Float.NaN, 2.0D, new BigDecimal("2.00"), "2", null, null)); DataFile dataFile = - FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); + FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); table.newAppend().appendFile(dataFile).commit(); return table; } @@ -144,12 +145,12 @@ private Pair createNestedTable() throws IOException { createNestedRecord(1L, Double.NaN), createNestedRecord(null, null)); DataFile dataFile = - FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); + FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); table.newAppend().appendFile(dataFile).commit(); return Pair.of(table, dataFile); } - @After + @AfterEach public void dropTable() { sql("DROP TABLE %s", tableName); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java index 5baf6071233d..1b64e062b1d0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java @@ -19,9 +19,11 @@ package org.apache.iceberg.spark.source; 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; +import java.nio.file.Path; import org.apache.iceberg.BaseTable; import org.apache.iceberg.Schema; import org.apache.iceberg.hadoop.HadoopTableOperations; @@ -29,40 +31,38 @@ import org.apache.iceberg.spark.PathIdentifier; import org.apache.iceberg.spark.SparkCatalog; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; import org.apache.spark.sql.connector.expressions.Transform; 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.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; -public class TestPathIdentifier extends SparkTestBase { +public class TestPathIdentifier extends TestBase { private static final Schema SCHEMA = new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; private File tableLocation; private PathIdentifier identifier; private SparkCatalog sparkCatalog; - @Before + @BeforeEach public void before() throws IOException { - tableLocation = temp.newFolder(); + tableLocation = temp.toFile(); identifier = new PathIdentifier(tableLocation.getAbsolutePath()); sparkCatalog = new SparkCatalog(); sparkCatalog.initialize("test", new CaseInsensitiveStringMap(ImmutableMap.of())); } - @After + @AfterEach public void after() { tableLocation.delete(); sparkCatalog = null; @@ -75,12 +75,12 @@ public void testPathIdentifier() throws TableAlreadyExistsException, NoSuchTable sparkCatalog.createTable( identifier, SparkSchemaUtil.convert(SCHEMA), new Transform[0], ImmutableMap.of()); - Assert.assertEquals(table.table().location(), tableLocation.getAbsolutePath()); - Assertions.assertThat(table.table()).isInstanceOf(BaseTable.class); - Assertions.assertThat(((BaseTable) table.table()).operations()) + assertThat(table.table().location()).isEqualTo(tableLocation.getAbsolutePath()); + assertThat(table.table()).isInstanceOf(BaseTable.class); + assertThat(((BaseTable) table.table()).operations()) .isInstanceOf(HadoopTableOperations.class); - Assert.assertEquals(sparkCatalog.loadTable(identifier), table); - Assert.assertTrue(sparkCatalog.dropTable(identifier)); + assertThat(sparkCatalog.loadTable(identifier)).isEqualTo(table); + assertThat(sparkCatalog.dropTable(identifier)).isTrue(); } } 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 eecc405b1a09..1b8a961ab187 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 @@ -19,8 +19,12 @@ package org.apache.iceberg.spark.source; import static org.apache.avro.Schema.Type.UNION; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +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.Schema; @@ -33,11 +37,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public abstract class TestReadProjection { final String format; @@ -49,7 +50,8 @@ public abstract class TestReadProjection { protected abstract Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; @Test public void testFullProjection() throws Exception { @@ -64,12 +66,13 @@ public void testFullProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, schema, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); + assertThat(34L).as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); - Assert.assertEquals("Should contain the correct data value", 0, cmp); + assertThat(0).as("Should contain the correct data value") + .isEqualTo(cmp); } @Test @@ -94,8 +97,9 @@ public void testReorderedFullProjection() throws Exception { Record projected = writeAndRead("reordered_full_projection", schema, reordered, record); - Assert.assertEquals("Should contain the correct 0 value", "test", projected.get(0).toString()); - Assert.assertEquals("Should contain the correct 1 value", 34L, projected.get(1)); + assertThat("test").as("Should contain the correct 0 value").isEqualTo(projected.get(0).toString()); + assertThat(34L).as("Should contain the correct 1 value").isEqualTo(projected.get(1)); + } @Test @@ -121,9 +125,9 @@ public void testReorderedProjection() throws Exception { Record projected = writeAndRead("reordered_projection", schema, reordered, record); - Assert.assertNull("Should contain the correct 0 value", projected.get(0)); - Assert.assertEquals("Should contain the correct 1 value", "test", projected.get(1).toString()); - Assert.assertNull("Should contain the correct 2 value", projected.get(2)); + assertThat(projected.get(0)).as("Should contain the correct 0 value").isNull(); + assertThat("test").as("Should contain the correct 1 value").isEqualTo(projected.get(1).toString()); + assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); } @Test @@ -139,9 +143,9 @@ public void testEmptyProjection() throws Exception { Record projected = writeAndRead("empty_projection", schema, schema.select(), record); - Assert.assertNotNull("Should read a non-null record", projected); + assertThat(projected).as("Should read a non-null record").isNotNull(); // this is expected because there are no values - Assertions.assertThatThrownBy(() -> projected.get(0)) + assertThatThrownBy(() -> projected.get(0)) .isInstanceOf(ArrayIndexOutOfBoundsException.class); } @@ -159,18 +163,17 @@ public void testBasicProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); - Assert.assertNull("Should not project data", projected.getField("data")); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); + assertThat(projected.getField("data")).as("Should not project data").isNull(); + assertThat(34L).as("Should contain the correct id value").isEqualTo((long) projected.getField("id")); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); - Assert.assertNull("Should not project id", projected.getField("id")); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); - Assert.assertEquals("Should contain the correct data value", 0, cmp); + assertThat(0).as("Should contain the correct data value").isEqualTo(cmp); } @Test @@ -190,12 +193,11 @@ public void testRename() throws Exception { Types.NestedField.optional(1, "renamed", Types.StringType.get())); Record projected = writeAndRead("project_and_rename", writeSchema, readSchema, record); + assertThat(34L).as("Should contain the correct id value").isEqualTo((long) projected.getField("id")); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("renamed")); - Assert.assertEquals("Should contain the correct data/renamed value", 0, cmp); + assertThat(0).as("Should contain the correct data/renamed value").isEqualTo(cmp); } @Test @@ -221,9 +223,8 @@ public void testNestedStructProjection() throws Exception { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Record projectedLocation = (Record) projected.getField("location"); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project location", projectedLocation); + assertThat(34L).as("Should contain the correct id value").isEqualTo((long) projected.getField("id")); + assertThat(projectedLocation).as("Should not project location").isNull(); Schema latOnly = new Schema( @@ -234,14 +235,12 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("latitude_only", writeSchema, latOnly, record); projectedLocation = (Record) projected.getField("location"); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project location", projected.getField("location")); - Assert.assertNull("Should not project longitude", projectedLocation.getField("long")); - Assert.assertEquals( - "Should project latitude", - 52.995143f, - (float) projectedLocation.getField("lat"), - 0.000001f); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("location")).as("Should project location").isNotNull(); + assertThat(projectedLocation.getField("long")).as("Should not project longitude").isNull(); + assertThat(52.995143f) + .as("Should project latitude") + .isCloseTo((float) projectedLocation.getField("lat"), within(0.000001f)); Schema longOnly = new Schema( @@ -252,30 +251,26 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = (Record) projected.getField("location"); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project location", projected.getField("location")); - Assert.assertNull("Should not project latitutde", projectedLocation.getField("lat")); - Assert.assertEquals( - "Should project longitude", - -1.539054f, - (float) projectedLocation.getField("long"), - 0.000001f); + + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("location")).as("Should project location").isNotNull(); + assertThat(projectedLocation.getField("lat")).as("Should not project latitude").isNull(); + assertThat(-1.539054f) + .as("Should project longitude") + .isCloseTo((float) projectedLocation.getField("long"), within(0.000001f)); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = (Record) projected.getField("location"); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project location", projected.getField("location")); - Assert.assertEquals( - "Should project latitude", - 52.995143f, - (float) projectedLocation.getField("lat"), - 0.000001f); - Assert.assertEquals( - "Should project longitude", - -1.539054f, - (float) projectedLocation.getField("long"), - 0.000001f); + + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("location")).as("Should project location").isNotNull(); + assertThat(52.995143f) + .as("Should project latitude") + .isCloseTo((float) projectedLocation.getField("lat"), within(0.000001f)); + assertThat(-1.539054f) + .as("Should project longitude") + .isCloseTo((float) projectedLocation.getField("long"), within(0.000001f)); } @Test @@ -297,33 +292,26 @@ public void testMapProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project properties map", projected.getField("properties")); + assertThat(34L).as("Should contain the correct id value").isEqualTo((long) projected.getField("id")); + assertThat(projected.getField("properties")).as("Should not project properties map").isNull(); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project entire map", - properties, - toStringMap((Map) projected.getField("properties"))); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(properties).as("Should project entire map") + .isEqualTo(toStringMap((Map) projected.getField("properties"))); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project entire map", - properties, - toStringMap((Map) projected.getField("properties"))); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(properties).as("Should project entire map") + .isEqualTo(toStringMap((Map) projected.getField("properties"))); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project entire map", - properties, - toStringMap((Map) projected.getField("properties"))); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(properties).as("Should project entire map") + .isEqualTo(toStringMap((Map) projected.getField("properties"))); } private Map toStringMap(Map map) { @@ -367,51 +355,59 @@ public void testMapOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project locations map", projected.getField("locations")); + assertThat(34L).as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); + assertThat(projected.getField("locations")).as("Should not project locations map").isNull(); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project locations map", - record.getField("locations"), - toStringMap((Map) projected.getField("locations"))); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(record.getField("locations")).as("Should project locations map") + .isEqualTo(toStringMap((Map) projected.getField("locations"))); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); - Assert.assertNull("Should not project id", projected.getField("id")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + Map locations = toStringMap((Map) projected.getField("locations")); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals( - "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(Sets.newHashSet("L1", "L2")).as("Should contain L1 and L2") + .isEqualTo(locations.keySet()); + Record projectedL1 = (Record) locations.get("L1"); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain lat", 53.992811f, (float) projectedL1.getField("lat"), 0.000001); - Assert.assertNull("L1 should not contain long", projectedL1.getField("long")); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat(53.992811f) + .as("L1 should contain lat") + .isCloseTo((float) projectedL1.getField("lat"), within(0.000001f)); + assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); + Record projectedL2 = (Record) locations.get("L2"); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain lat", 52.995143f, (float) projectedL2.getField("lat"), 0.000001); - Assert.assertNull("L2 should not contain long", projectedL2.getField("long")); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat(52.995143f) + .as("L2 should contain lat") + .isCloseTo((float) projectedL2.getField("lat"), within(0.000001f)); + assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), record); - Assert.assertNull("Should not project id", projected.getField("id")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + locations = toStringMap((Map) projected.getField("locations")); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals( - "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(Sets.newHashSet("L1", "L2")).as("Should contain L1 and L2") + .isEqualTo(locations.keySet()); + projectedL1 = (Record) locations.get("L1"); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertNull("L1 should not contain lat", projectedL1.getField("lat")); - Assert.assertEquals( - "L1 should contain long", -1.542616f, (float) projectedL1.getField("long"), 0.000001); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); + assertThat(-1.542616f) + .as("L1 should contain long") + .isCloseTo((float) projectedL1.getField("long"), within(0.000001f)); + projectedL2 = (Record) locations.get("L2"); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertNull("L2 should not contain lat", projectedL2.getField("lat")); - Assert.assertEquals( - "L2 should contain long", -1.539054f, (float) projectedL2.getField("long"), 0.000001); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); + assertThat(-1.539054f) + .as("L2 should contain long") + .isCloseTo((float) projectedL2.getField("long"), within(0.000001f)); Schema latitiudeRenamed = new Schema( @@ -426,29 +422,27 @@ public void testMapOfStructsProjection() throws IOException { Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, record); - Assert.assertNull("Should not project id", projected.getField("id")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); locations = toStringMap((Map) projected.getField("locations")); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals( - "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(Sets.newHashSet("L1", "L2")).as("Should contain L1 and L2") + .isEqualTo(locations.keySet()); + projectedL1 = (Record) locations.get("L1"); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain latitude", - 53.992811f, - (float) projectedL1.getField("latitude"), - 0.000001); - Assert.assertNull("L1 should not contain lat", projectedL1.getField("lat")); - Assert.assertNull("L1 should not contain long", projectedL1.getField("long")); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat(53.992811f) + .as("L1 should contain latitude") + .isCloseTo((float) projectedL1.getField("latitude"), within(0.000001f)); + assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); + assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); + projectedL2 = (Record) locations.get("L2"); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain latitude", - 52.995143f, - (float) projectedL2.getField("latitude"), - 0.000001); - Assert.assertNull("L2 should not contain lat", projectedL2.getField("lat")); - Assert.assertNull("L2 should not contain long", projectedL2.getField("long")); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat(52.995143f) + .as("L2 should contain latitude") + .isCloseTo((float) projectedL2.getField("latitude"), within(0.000001f)); + assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); + assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); } @Test @@ -468,19 +462,19 @@ public void testListProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project values list", projected.getField("values")); + assertThat(34L).as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); + assertThat(projected.getField("values")).as("Should not project values list").isNull(); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals("Should project entire list", values, projected.getField("values")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(values).as("Should project entire list").isEqualTo(projected.getField("values")); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals("Should project entire list", values, projected.getField("values")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(values).as("Should project entire list").isEqualTo(projected.getField("values")); } @Test @@ -511,38 +505,44 @@ public void testListOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project points list", projected.getField("points")); + assertThat(34L).as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); + assertThat(projected.getField("points")).as("Should not project points list").isNull(); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project points list", record.getField("points"), projected.getField("points")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(record.getField("points")).as("Should project points list") + .isEqualTo(projected.getField("points")); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project points list", projected.getField("points")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + List points = (List) projected.getField("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); + Record projectedP1 = points.get(0); - Assert.assertEquals("Should project x", 1, (int) projectedP1.getField("x")); - Assert.assertNull("Should not project y", projectedP1.getField("y")); + assertThat(1).as("Should project x").isEqualTo((int) projectedP1.getField("x")); + assertThat(projected.getField("y")).as("Should not project y").isNull(); + Record projectedP2 = points.get(1); - Assert.assertEquals("Should project x", 3, (int) projectedP2.getField("x")); - Assert.assertNull("Should not project y", projectedP2.getField("y")); + assertThat(3).as("Should project x").isEqualTo((int) projectedP2.getField("x")); + assertThat(projected.getField("y")).as("Should not project y").isNull(); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project points list", projected.getField("points")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + points = (List) projected.getField("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); + projectedP1 = points.get(0); - Assert.assertNull("Should not project x", projectedP1.getField("x")); - Assert.assertEquals("Should project y", 2, (int) projectedP1.getField("y")); + assertThat(projectedP1.getField("x")).as("Should not project x").isNull(); + assertThat(2).as("Should project y").isEqualTo((int) projectedP1.getField("y")); + projectedP2 = points.get(1); - Assert.assertNull("Should not project x", projectedP2.getField("x")); - Assert.assertNull("Should project null y", projectedP2.getField("y")); + assertThat(projectedP2.getField("x")).as("Should not project x").isNull(); + assertThat(projectedP2.getField("y")).as("Should not project y").isNull(); Schema yRenamed = new Schema( @@ -555,18 +555,21 @@ public void testListOfStructsProjection() throws IOException { Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); projected = writeAndRead("y_renamed", writeSchema, yRenamed, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project points list", projected.getField("points")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + points = (List) projected.getField("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); + projectedP1 = points.get(0); - Assert.assertNull("Should not project x", projectedP1.getField("x")); - Assert.assertNull("Should not project y", projectedP1.getField("y")); - Assert.assertEquals("Should project z", 2, (int) projectedP1.getField("z")); + assertThat(projectedP1.getField("x")).as("Should not project x").isNull(); + assertThat(projectedP1.getField("y")).as("Should not project y").isNull(); + assertThat(2).as("Should project z").isEqualTo((int) projectedP1.getField("z")); + projectedP2 = points.get(1); - Assert.assertNull("Should not project x", projectedP2.getField("x")); - Assert.assertNull("Should not project y", projectedP2.getField("y")); - Assert.assertNull("Should project null z", projectedP2.getField("z")); + assertThat(projectedP2.getField("x")).as("Should not project x").isNull(); + assertThat(projectedP2.getField("y")).as("Should not project y").isNull(); + assertThat(projectedP2.getField("z")).as("Should project null z").isNull(); Schema zAdded = new Schema( @@ -581,18 +584,21 @@ public void testListOfStructsProjection() throws IOException { Types.NestedField.optional(20, "z", Types.IntegerType.get()))))); projected = writeAndRead("z_added", writeSchema, zAdded, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project points list", projected.getField("points")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + points = (List) projected.getField("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); + projectedP1 = points.get(0); - Assert.assertEquals("Should project x", 1, (int) projectedP1.getField("x")); - Assert.assertEquals("Should project y", 2, (int) projectedP1.getField("y")); - Assert.assertNull("Should contain null z", projectedP1.getField("z")); + assertThat(1).as("Should project x").isEqualTo((int) projectedP1.getField("x")); + assertThat(2).as("Should project y").isEqualTo((int) projectedP1.getField("y")); + assertThat(projectedP1.getField("z")).as("Should contain null z").isNull(); + projectedP2 = points.get(1); - Assert.assertEquals("Should project x", 3, (int) projectedP2.getField("x")); - Assert.assertNull("Should project null y", projectedP2.getField("y")); - Assert.assertNull("Should contain null z", projectedP2.getField("z")); + assertThat(3).as("Should project x").isEqualTo((int) projectedP2.getField("x")); + assertThat(projectedP2.getField("y")).as("Should project null y").isNull(); + assertThat(projectedP2.getField("z")).as("Should contain null z").isNull(); } private static org.apache.avro.Schema fromOption(org.apache.avro.Schema schema) { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java index e2d6f744f5a5..2846333234e4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; import org.apache.iceberg.expressions.Expression; @@ -29,8 +30,7 @@ import org.apache.spark.sql.connector.expressions.aggregate.CountStar; import org.apache.spark.sql.connector.expressions.aggregate.Max; import org.apache.spark.sql.connector.expressions.aggregate.Min; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSparkAggregates { @@ -50,27 +50,26 @@ public void testAggregates() { Max max = new Max(namedReference); Expression expectedMax = Expressions.max(unquoted); Expression actualMax = SparkAggregates.convert(max); - Assert.assertEquals("Max must match", expectedMax.toString(), actualMax.toString()); + assertThat(expectedMax.toString()).as("Max must match").isEqualTo(String.valueOf(actualMax)); Min min = new Min(namedReference); Expression expectedMin = Expressions.min(unquoted); Expression actualMin = SparkAggregates.convert(min); - Assert.assertEquals("Min must match", expectedMin.toString(), actualMin.toString()); + assertThat(expectedMin.toString()).as("Min must match").isEqualTo(String.valueOf(actualMin)); Count count = new Count(namedReference, false); Expression expectedCount = Expressions.count(unquoted); Expression actualCount = SparkAggregates.convert(count); - Assert.assertEquals("Count must match", expectedCount.toString(), actualCount.toString()); + assertThat(expectedCount.toString()).as("Count must match").isEqualTo(String.valueOf(actualCount)); Count countDistinct = new Count(namedReference, true); Expression convertedCountDistinct = SparkAggregates.convert(countDistinct); - Assert.assertNull("Count Distinct is converted to null", convertedCountDistinct); + assertThat(convertedCountDistinct).as("Count Distinct is converted to null").isNull(); CountStar countStar = new CountStar(); Expression expectedCountStar = Expressions.countStar(); Expression actualCountStar = SparkAggregates.convert(countStar); - Assert.assertEquals( - "CountStar must match", expectedCountStar.toString(), actualCountStar.toString()); + assertThat(expectedCountStar.toString()).as("CountStar must match").isEqualTo(String.valueOf(actualCountStar)); }); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java index 3d668197fd51..7911ce0d966b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; import org.apache.iceberg.CachingCatalog; @@ -25,13 +26,12 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.spark.SparkCatalog; import org.apache.iceberg.spark.SparkSessionCatalog; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.assertj.core.api.Assertions; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; -public class TestSparkCatalogCacheExpiration extends SparkTestBaseWithCatalog { +public class TestSparkCatalogCacheExpiration extends TestBaseWithCatalog { private static final String sessionCatalogName = "spark_catalog"; private static final String sessionCatalogImpl = SparkSessionCatalog.class.getName(); @@ -57,7 +57,7 @@ private static String asSqlConfCatalogKeyFor(String catalog, String configKey) { // Add more catalogs to the spark session, so we only need to start spark one time for multiple // different catalog configuration tests. - @BeforeClass + @BeforeAll public static void beforeClass() { // Catalog - expiration_disabled: Catalog with caching on and expiration disabled. ImmutableMap.of( @@ -93,18 +93,18 @@ public TestSparkCatalogCacheExpiration() { @Test public void testSparkSessionCatalogWithExpirationEnabled() { SparkSessionCatalog sparkCatalog = sparkSessionCatalog(); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .extracting("cacheEnabled") .isEqualTo(true); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .extracting("icebergCatalog") .isInstanceOfSatisfying( Catalog.class, icebergCatalog -> { - Assertions.assertThat(icebergCatalog) + assertThat(icebergCatalog) .isExactlyInstanceOf(CachingCatalog.class) .extracting("expirationIntervalMillis") .isEqualTo(3000L); @@ -114,14 +114,14 @@ public void testSparkSessionCatalogWithExpirationEnabled() { @Test public void testCacheEnabledAndExpirationDisabled() { SparkCatalog sparkCatalog = getSparkCatalog("expiration_disabled"); - Assertions.assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(true); + assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(true); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .isInstanceOfSatisfying( CachingCatalog.class, icebergCatalog -> { - Assertions.assertThat(icebergCatalog) + assertThat(icebergCatalog) .extracting("expirationIntervalMillis") .isEqualTo(-1L); }); @@ -130,14 +130,14 @@ public void testCacheEnabledAndExpirationDisabled() { @Test public void testCacheDisabledImplicitly() { SparkCatalog sparkCatalog = getSparkCatalog("cache_disabled_implicitly"); - Assertions.assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(false); + assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(false); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .isInstanceOfSatisfying( Catalog.class, icebergCatalog -> - Assertions.assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); + assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); } private SparkSessionCatalog sparkSessionCatalog() { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index b894d32326dc..23949e114675 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -25,6 +25,7 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -70,12 +71,11 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestSparkDataFile { @@ -119,13 +119,13 @@ public class TestSparkDataFile { private static SparkSession spark; private static JavaSparkContext sparkContext = null; - @BeforeClass + @BeforeAll public static void startSpark() { TestSparkDataFile.spark = SparkSession.builder().master("local[2]").getOrCreate(); TestSparkDataFile.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestSparkDataFile.spark; TestSparkDataFile.spark = null; @@ -133,12 +133,13 @@ public static void stopSpark() { currentSpark.stop(); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + private Path temp; private String tableLocation = null; - @Before + @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = temp.newFolder(); + File tableDir = temp.toFile(); this.tableLocation = tableDir.toURI().toString(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java index fea5a1d3e1c3..4ee2d9e4e558 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -18,24 +18,24 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; import static scala.collection.JavaConverters.seqAsJavaListConverter; import java.util.List; import java.util.Map; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +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.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.execution.metric.SQLMetric; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import scala.collection.JavaConverters; -public class TestSparkReadMetrics extends SparkTestBaseWithCatalog { +public class TestSparkReadMetrics extends TestBaseWithCatalog { - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } @@ -57,30 +57,30 @@ public void testReadMetricsForV1Table() throws NoSuchTableException { Map metricsMap = JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); // Common - Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); // data manifests - Assertions.assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); + assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); + assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); // data files - Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); + assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); // delete manifests - Assertions.assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); // delete files - Assertions.assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); + assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); } @Test @@ -101,30 +101,30 @@ public void testReadMetricsForV2Table() throws NoSuchTableException { JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); // Common - Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); // data manifests - Assertions.assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); + assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); + assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); // data files - Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); + assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); // delete manifests - Assertions.assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); // delete files - Assertions.assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); + assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); } @Test @@ -152,29 +152,29 @@ public void testDeleteMetrics() throws NoSuchTableException { JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); // Common - Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); // data manifests - Assertions.assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(1); + assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); // data files - Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); + assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); // delete manifests - Assertions.assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(1); + assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); // delete files - Assertions.assertThat(metricsMap.get("totalDeleteFileSize").value()).isNotEqualTo(0); - Assertions.assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteFileSize").value()).isNotEqualTo(0); + assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); } } 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..50e6bf7364e8 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 @@ -110,7 +110,7 @@ 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 = temp.resolve(desc).toFile(); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); Assert.assertTrue("mkdirs should succeed", dataFolder.mkdirs()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java index 241293f367aa..b8830b611e4e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.util.List; @@ -26,25 +27,23 @@ import org.apache.iceberg.Table; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.ScanTaskSetManager; -import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkReadOptions; 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.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; -public class TestSparkStagedScan extends SparkCatalogTestBase { +public class TestSparkStagedScan extends CatalogTestBase { public TestSparkStagedScan( String catalogName, String implementation, Map config) { super(catalogName, implementation, config); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } @@ -59,7 +58,7 @@ public void testTaskSetLoading() throws NoSuchTableException, IOException { df.writeTo(tableName).append(); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should produce 1 snapshot", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should produce 1 snapshot").hasSize(1); try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); @@ -95,7 +94,7 @@ public void testTaskSetPlanning() throws NoSuchTableException, IOException { df.coalesce(1).writeTo(tableName).append(); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should produce 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should produce 2 snapshot").hasSize(2); try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); @@ -111,7 +110,7 @@ public void testTaskSetPlanning() throws NoSuchTableException, IOException { .option(SparkReadOptions.SCAN_TASK_SET_ID, setID) .option(SparkReadOptions.SPLIT_SIZE, tasks.get(0).file().fileSizeInBytes()) .load(tableName); - Assert.assertEquals("Num partitions should match", 2, scanDF.javaRDD().getNumPartitions()); + assertThat(2).as("Num partitions should match").isEqualTo(scanDF.javaRDD().getNumPartitions()); // load the staged file set and make sure we combine both files into a single split scanDF = @@ -121,7 +120,7 @@ public void testTaskSetPlanning() throws NoSuchTableException, IOException { .option(SparkReadOptions.SCAN_TASK_SET_ID, setID) .option(SparkReadOptions.SPLIT_SIZE, Long.MAX_VALUE) .load(tableName); - Assert.assertEquals("Num partitions should match", 1, scanDF.javaRDD().getNumPartitions()); + assertThat(1).as("Num partitions should match").isEqualTo(scanDF.javaRDD().getNumPartitions()); } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java index 616a196872de..621ac0de2df2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java @@ -17,30 +17,30 @@ * under the License. */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.CatalogManager; 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.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; -public class TestSparkTable extends SparkCatalogTestBase { +public class TestSparkTable extends CatalogTestBase { public TestSparkTable(String catalogName, String implementation, Map config) { super(catalogName, implementation, config); } - @Before + @BeforeEach public void createTable() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); } - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); } @@ -54,7 +54,7 @@ public void testTableEquality() throws NoSuchTableException { SparkTable table2 = (SparkTable) catalog.loadTable(identifier); // different instances pointing to the same table must be equivalent - Assert.assertNotSame("References must be different", table1, table2); - Assert.assertEquals("Tables must be equivalent", table1, table2); + assertThat(table1).as("References must be different").isNotSameAs(table2); + assertThat(table1).as("Tables must be equivalent").isEqualTo(table2); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java index 17370aaa22f2..df6308d7ce8b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java @@ -17,12 +17,12 @@ * under the License. */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; import com.fasterxml.jackson.databind.node.ObjectNode; import java.util.Arrays; import org.apache.iceberg.util.JsonUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestStreamingOffset { @@ -35,10 +35,8 @@ public void testJsonConversion() { new StreamingOffset(System.currentTimeMillis(), 3L, false), new StreamingOffset(System.currentTimeMillis(), 4L, true) }; - Assert.assertArrayEquals( - "StreamingOffsets should match", - expected, - Arrays.stream(expected).map(elem -> StreamingOffset.fromJson(elem.json())).toArray()); + assertThat(expected).as("StreamingOffsets should match") + .isEqualTo(Arrays.stream(expected).map(elem -> StreamingOffset.fromJson(elem.json())).toArray()); } @Test @@ -51,6 +49,6 @@ public void testToJson() throws Exception { actual.put("scan_all_files", false); String expectedJson = expected.json(); String actualJson = JsonUtil.mapper().writeValueAsString(actual); - Assert.assertEquals("Json should match", expectedJson, actualJson); + assertThat(expectedJson).isEqualTo(actualJson); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index 464f1f5922b3..e49d68f643d9 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -19,15 +19,17 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; +import java.nio.file.Path; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; @@ -40,13 +42,10 @@ import org.apache.spark.sql.streaming.DataStreamWriter; import org.apache.spark.sql.streaming.StreamingQuery; import org.apache.spark.sql.streaming.StreamingQueryException; -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.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import scala.Option; import scala.collection.JavaConverters; @@ -58,9 +57,10 @@ public class TestStructuredStreaming { optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); private static SparkSession spark = null; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; - @BeforeClass + @BeforeAll public static void startSpark() { TestStructuredStreaming.spark = SparkSession.builder() @@ -69,7 +69,7 @@ public static void startSpark() { .getOrCreate(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestStructuredStreaming.spark; TestStructuredStreaming.spark = null; @@ -78,7 +78,7 @@ public static void stopSpark() { @Test public void testStreamingWriteAppendMode() throws Exception { - File parent = temp.newFolder("parquet"); + File parent = temp.resolve("parquet").toFile(); File location = new File(parent, "test-table"); File checkpoint = new File(parent, "checkpoint"); @@ -117,7 +117,7 @@ public void testStreamingWriteAppendMode() throws Exception { // remove the last commit to force Spark to reprocess batch #1 File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); - Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); + assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue(); // restart the query from the checkpoint StreamingQuery restartedQuery = streamWriter.start(); @@ -127,9 +127,10 @@ public void testStreamingWriteAppendMode() throws Exception { Dataset result = spark.read().format("iceberg").load(location.toString()); 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); - Assert.assertEquals("Number of snapshots should match", 2, Iterables.size(table.snapshots())); + + assertThat(expected).as("Number of rows should match").hasSameSizeAs(actual); + assertThat(expected).as("Result rows should match").isEqualTo(actual); + assertThat(table.snapshots()).as("Number of snapshots should match").hasSize(2); } finally { for (StreamingQuery query : spark.streams().active()) { query.stop(); @@ -139,7 +140,7 @@ public void testStreamingWriteAppendMode() throws Exception { @Test public void testStreamingWriteCompleteMode() throws Exception { - File parent = temp.newFolder("parquet"); + File parent = temp.resolve("parquet").toFile(); File location = new File(parent, "test-table"); File checkpoint = new File(parent, "checkpoint"); @@ -177,7 +178,7 @@ public void testStreamingWriteCompleteMode() throws Exception { // remove the last commit to force Spark to reprocess batch #1 File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); - Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); + assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue(); // restart the query from the checkpoint StreamingQuery restartedQuery = streamWriter.start(); @@ -187,9 +188,10 @@ public void testStreamingWriteCompleteMode() throws Exception { Dataset result = spark.read().format("iceberg").load(location.toString()); List actual = result.orderBy("data").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); - Assert.assertEquals("Number of snapshots should match", 2, Iterables.size(table.snapshots())); + + assertThat(expected).as("Number of rows should match").hasSameSizeAs(actual); + assertThat(expected).as("Result rows should match").isEqualTo(actual); + assertThat(table.snapshots()).as("Number of snapshots should match").hasSize(2); } finally { for (StreamingQuery query : spark.streams().active()) { query.stop(); @@ -199,7 +201,7 @@ public void testStreamingWriteCompleteMode() throws Exception { @Test public void testStreamingWriteCompleteModeWithProjection() throws Exception { - File parent = temp.newFolder("parquet"); + File parent = temp.resolve("parquet").toFile(); File location = new File(parent, "test-table"); File checkpoint = new File(parent, "checkpoint"); @@ -237,7 +239,7 @@ public void testStreamingWriteCompleteModeWithProjection() throws Exception { // remove the last commit to force Spark to reprocess batch #1 File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); - Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); + assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue(); // restart the query from the checkpoint StreamingQuery restartedQuery = streamWriter.start(); @@ -247,9 +249,10 @@ public void testStreamingWriteCompleteModeWithProjection() throws Exception { Dataset result = spark.read().format("iceberg").load(location.toString()); 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); - Assert.assertEquals("Number of snapshots should match", 2, Iterables.size(table.snapshots())); + + assertThat(expected).as("Number of rows should match").hasSameSizeAs(actual); + assertThat(expected).as("Result rows should match").isEqualTo(actual); + assertThat(table.snapshots()).as("Number of snapshots should match").hasSize(2); } finally { for (StreamingQuery query : spark.streams().active()) { query.stop(); @@ -259,7 +262,7 @@ public void testStreamingWriteCompleteModeWithProjection() throws Exception { @Test public void testStreamingWriteUpdateMode() throws Exception { - File parent = temp.newFolder("parquet"); + File parent = temp.resolve("parquet").toFile(); File location = new File(parent, "test-table"); File checkpoint = new File(parent, "checkpoint"); @@ -283,7 +286,7 @@ public void testStreamingWriteUpdateMode() throws Exception { List batch1 = Lists.newArrayList(1, 2); send(batch1, inputStream); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .isInstanceOf(StreamingQueryException.class) .hasMessageContaining("does not support Update mode"); } finally { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java index 1e2a825d8e76..4755331e4dc8 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java @@ -21,9 +21,12 @@ import static org.apache.iceberg.spark.SparkSchemaUtil.convert; 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.Assertions.assertThatThrownBy; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -48,13 +51,10 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; -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.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestWriteMetricsConfig { @@ -73,18 +73,19 @@ public class TestWriteMetricsConfig { required(4, "id", Types.IntegerType.get()), required(5, "data", Types.StringType.get())))); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + public Path temp; private static SparkSession spark = null; private static JavaSparkContext sc = null; - @BeforeClass + @BeforeAll public static void startSpark() { TestWriteMetricsConfig.spark = SparkSession.builder().master("local[2]").getOrCreate(); TestWriteMetricsConfig.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestWriteMetricsConfig.spark; TestWriteMetricsConfig.spark = null; @@ -94,7 +95,7 @@ public static void stopSpark() { @Test public void testFullMetricsCollectionForParquet() 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(); @@ -116,16 +117,17 @@ public void testFullMetricsCollectionForParquet() throws IOException { for (FileScanTask task : table.newScan().includeColumnStats().planFiles()) { DataFile file = task.file(); - Assert.assertEquals(2, file.nullValueCounts().size()); - Assert.assertEquals(2, file.valueCounts().size()); - Assert.assertEquals(2, file.lowerBounds().size()); - Assert.assertEquals(2, file.upperBounds().size()); + + assertThat(file.nullValueCounts()).hasSize(2); + assertThat(file.valueCounts()).hasSize(2); + assertThat(file.lowerBounds()).hasSize(2); + assertThat(file.upperBounds()).hasSize(2); } } @Test public void testCountMetricsCollectionForParquet() 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(); @@ -147,16 +149,16 @@ public void testCountMetricsCollectionForParquet() throws IOException { for (FileScanTask task : table.newScan().includeColumnStats().planFiles()) { DataFile file = task.file(); - Assert.assertEquals(2, file.nullValueCounts().size()); - Assert.assertEquals(2, file.valueCounts().size()); - Assert.assertTrue(file.lowerBounds().isEmpty()); - Assert.assertTrue(file.upperBounds().isEmpty()); + assertThat(file.nullValueCounts()).hasSize(2); + assertThat(file.valueCounts()).hasSize(2); + assertThat(file.lowerBounds()).isEmpty(); + assertThat(file.upperBounds()).isEmpty(); } } @Test public void testNoMetricsCollectionForParquet() 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(); @@ -178,16 +180,16 @@ public void testNoMetricsCollectionForParquet() throws IOException { for (FileScanTask task : table.newScan().includeColumnStats().planFiles()) { DataFile file = task.file(); - Assert.assertTrue(file.nullValueCounts().isEmpty()); - Assert.assertTrue(file.valueCounts().isEmpty()); - Assert.assertTrue(file.lowerBounds().isEmpty()); - Assert.assertTrue(file.upperBounds().isEmpty()); + assertThat(file.nullValueCounts()).isEmpty(); + assertThat(file.valueCounts()).isEmpty(); + assertThat(file.lowerBounds()).isEmpty(); + assertThat(file.upperBounds()).isEmpty(); } } @Test public void testCustomMetricCollectionForParquet() 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(); @@ -212,18 +214,18 @@ public void testCustomMetricCollectionForParquet() throws IOException { Types.NestedField id = schema.findField("id"); for (FileScanTask task : table.newScan().includeColumnStats().planFiles()) { DataFile file = task.file(); - Assert.assertEquals(2, file.nullValueCounts().size()); - Assert.assertEquals(2, file.valueCounts().size()); - Assert.assertEquals(1, file.lowerBounds().size()); - Assert.assertTrue(file.lowerBounds().containsKey(id.fieldId())); - Assert.assertEquals(1, file.upperBounds().size()); - Assert.assertTrue(file.upperBounds().containsKey(id.fieldId())); + assertThat(file.nullValueCounts()).hasSize(2); + assertThat(file.valueCounts()).hasSize(2); + assertThat(file.lowerBounds()).hasSize(1); + assertThat(file.lowerBounds()).containsKey(id.fieldId()); + assertThat(file.upperBounds()).hasSize(1); + assertThat(file.upperBounds()).containsKey(id.fieldId()); } } @Test public void testBadCustomMetricCollectionForParquet() 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(); @@ -231,7 +233,7 @@ public void testBadCustomMetricCollectionForParquet() throws IOException { properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "counts"); properties.put("write.metadata.metrics.column.ids", "full"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith( @@ -240,7 +242,7 @@ public void testBadCustomMetricCollectionForParquet() throws IOException { @Test public void testCustomMetricCollectionForNestedParquet() 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.builderFor(COMPLEX_SCHEMA).identity("strCol").build(); @@ -271,28 +273,30 @@ public void testCustomMetricCollectionForNestedParquet() throws IOException { DataFile file = task.file(); Map nullValueCounts = file.nullValueCounts(); - Assert.assertEquals(3, nullValueCounts.size()); - Assert.assertTrue(nullValueCounts.containsKey(longCol.fieldId())); - Assert.assertTrue(nullValueCounts.containsKey(recordId.fieldId())); - Assert.assertTrue(nullValueCounts.containsKey(recordData.fieldId())); + assertThat(nullValueCounts).hasSize(3); + assertThat(nullValueCounts).containsKey(longCol.fieldId()); + assertThat(nullValueCounts).containsKey(recordId.fieldId()); + assertThat(nullValueCounts).containsKey(recordData.fieldId()); Map valueCounts = file.valueCounts(); - Assert.assertEquals(3, valueCounts.size()); - Assert.assertTrue(valueCounts.containsKey(longCol.fieldId())); - Assert.assertTrue(valueCounts.containsKey(recordId.fieldId())); - Assert.assertTrue(valueCounts.containsKey(recordData.fieldId())); + assertThat(valueCounts).hasSize(3); + assertThat(valueCounts).containsKey(longCol.fieldId()); + assertThat(valueCounts).containsKey(recordId.fieldId()); + assertThat(valueCounts).containsKey(recordData.fieldId()); Map lowerBounds = file.lowerBounds(); - Assert.assertEquals(2, lowerBounds.size()); - Assert.assertTrue(lowerBounds.containsKey(recordId.fieldId())); + assertThat(lowerBounds).hasSize(2); + assertThat(lowerBounds).containsKey(recordId.fieldId()); + ByteBuffer recordDataLowerBound = lowerBounds.get(recordData.fieldId()); - Assert.assertEquals(2, ByteBuffers.toByteArray(recordDataLowerBound).length); + assertThat(ByteBuffers.toByteArray(recordDataLowerBound)).hasSize(2); Map upperBounds = file.upperBounds(); - Assert.assertEquals(2, upperBounds.size()); - Assert.assertTrue(upperBounds.containsKey(recordId.fieldId())); + assertThat(upperBounds).hasSize(2); + assertThat(upperBounds).containsKey(recordId.fieldId()); + ByteBuffer recordDataUpperBound = upperBounds.get(recordData.fieldId()); - Assert.assertEquals(2, ByteBuffers.toByteArray(recordDataUpperBound).length); + assertThat(ByteBuffers.toByteArray(recordDataUpperBound)).hasSize(2); } } } From 30b5601dccce2220b6815a0d151c41206d6a59b0 Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Tue, 19 Dec 2023 20:53:54 +0530 Subject: [PATCH 02/15] update hasNext() --- .../iceberg/spark/source/TestIcebergSourceTablesBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 003e5e0769ed..135ed2d4a0af 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -702,7 +702,7 @@ public void testAllMetadataTablesWithStagedCommits() { .load(loadLocation(tableIdentifier, "all_entries")) .collectAsList(); - assertThat(table.snapshots().iterator().hasNext()).as("Stage table should have some snapshots").isTrue(); + assertThat(table.snapshots().iterator()).as("Stage table should have some snapshots").hasNext(); assertThat(table.currentSnapshot()).as("Stage table should have null currentSnapshot").isNull(); assertThat(actualAllData).as("Actual results should have two rows").hasSize(2); assertThat(actualAllManifests).as("Actual results should have two rows").hasSize(2); From 18df1cf048bf54d12510ce093c2d6b6a7d2dcbfe Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Wed, 20 Dec 2023 07:32:47 +0530 Subject: [PATCH 03/15] changed temp to private --- .../java/org/apache/iceberg/spark/source/TestBaseReader.java | 2 +- .../org/apache/iceberg/spark/source/TestChangelogReader.java | 2 +- .../org/apache/iceberg/spark/source/TestDataSourceOptions.java | 2 +- .../apache/iceberg/spark/source/TestForwardCompatibility.java | 2 +- .../iceberg/spark/source/TestIcebergSourceTablesBase.java | 2 +- .../iceberg/spark/source/TestMetadataTableReadableMetrics.java | 2 +- .../org/apache/iceberg/spark/source/TestPathIdentifier.java | 2 +- .../org/apache/iceberg/spark/source/TestReadProjection.java | 2 +- .../apache/iceberg/spark/source/TestStructuredStreaming.java | 2 +- .../org/apache/iceberg/spark/source/TestWriteMetricsConfig.java | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java index 6adf07c5cec5..3ec73a338046 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java @@ -55,7 +55,7 @@ public class TestBaseReader { @TempDir - public Path temp; + private Path temp; private Table table; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java index 39f228bd68cb..080ba7d59359 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java @@ -65,7 +65,7 @@ public class TestChangelogReader extends TestBase { private DataFile dataFile2; @TempDir - public Path temp; + private Path temp; @BeforeEach public void before() throws IOException { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 480319bf1f4e..12b6347282c6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -71,7 +71,7 @@ public class TestDataSourceOptions extends TestBaseWithCatalog { private static SparkSession spark = null; @TempDir - public Path temp; + private Path temp; @BeforeAll public static void startSpark() { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 5972ee4af1dd..314237f62a09 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -88,7 +88,7 @@ public class TestForwardCompatibility { .build(); @TempDir - public Path temp; + private Path temp; private static SparkSession spark = null; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 135ed2d4a0af..0cebf6c272d9 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -118,7 +118,7 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("id").build(); @TempDir - public Path temp; + private Path temp; public abstract Table createTable( TableIdentifier ident, Schema schema, PartitionSpec spec, Map properties); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java index cb6f20f58085..bc945209da87 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java @@ -54,7 +54,7 @@ public class TestMetadataTableReadableMetrics extends TestBaseWithCatalog { @TempDir - public Path temp; + private Path temp; private static final Types.StructType LEAF_STRUCT_TYPE = Types.StructType.of( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java index 1b64e062b1d0..01376890df6a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java @@ -49,7 +49,7 @@ public class TestPathIdentifier extends TestBase { required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); @TempDir - public Path temp; + private Path temp; private File tableLocation; private PathIdentifier identifier; private SparkCatalog sparkCatalog; 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 1b8a961ab187..44aafb9bf2fa 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 @@ -51,7 +51,7 @@ protected abstract Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; @TempDir - public Path temp; + private Path temp; @Test public void testFullProjection() throws Exception { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index e49d68f643d9..6d89a0e589f4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -58,7 +58,7 @@ public class TestStructuredStreaming { private static SparkSession spark = null; @TempDir - public Path temp; + private Path temp; @BeforeAll public static void startSpark() { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java index 4755331e4dc8..d1b09d090848 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java @@ -74,7 +74,7 @@ public class TestWriteMetricsConfig { required(5, "data", Types.StringType.get())))); @TempDir - public Path temp; + private Path temp; private static SparkSession spark = null; private static JavaSparkContext sc = null; From 645ee276c52bff49cb83550c0d79e9ff5a695585 Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Wed, 20 Dec 2023 07:33:44 +0530 Subject: [PATCH 04/15] run spotlessApply --- .../iceberg/spark/source/TestBaseReader.java | 39 +++-- .../spark/source/TestChangelogReader.java | 7 +- .../spark/source/TestDataFrameWriterV2.java | 4 +- .../spark/source/TestDataSourceOptions.java | 7 +- .../source/TestForwardCompatibility.java | 3 +- .../source/TestIcebergSourceTablesBase.java | 109 +++++++------- .../spark/source/TestIcebergSpark.java | 57 +++++--- .../TestMetadataTableReadableMetrics.java | 9 +- .../spark/source/TestPathIdentifier.java | 6 +- .../spark/source/TestReadProjection.java | 134 ++++++++++-------- .../spark/source/TestSparkAggregates.java | 17 ++- .../TestSparkCatalogCacheExpiration.java | 8 +- .../spark/source/TestSparkDataFile.java | 3 +- .../spark/source/TestSparkStagedScan.java | 9 +- .../iceberg/spark/source/TestSparkTable.java | 1 + .../spark/source/TestStreamingOffset.java | 7 +- .../spark/source/TestStructuredStreaming.java | 3 +- .../spark/source/TestWriteMetricsConfig.java | 8 +- 18 files changed, 239 insertions(+), 192 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java index 3ec73a338046..27e7d7c496ef 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java @@ -54,8 +54,7 @@ public class TestBaseReader { - @TempDir - private Path temp; + @TempDir private Path temp; private Table table; @@ -133,10 +132,14 @@ public void testClosureOnDataExhaustion() throws IOException { assertThat(reader.get()).as("Reader should return non-null value").isNotNull(); } - assertThat(totalTasks * recordPerTask).as("Reader returned incorrect number of records").isEqualTo(countRecords); + assertThat(totalTasks * recordPerTask) + .as("Reader returned incorrect number of records") + .isEqualTo(countRecords); tasks.forEach( t -> - assertThat(reader.isIteratorClosed(t)).as("All iterators should be closed after read exhausion").isTrue()); + assertThat(reader.isIteratorClosed(t)) + .as("All iterators should be closed after read exhausion") + .isTrue()); } @Test @@ -152,11 +155,17 @@ public void testClosureDuringIteration() throws IOException { // Total of 2 elements assertThat(reader.next()).isTrue(); - assertThat(reader.isIteratorClosed(firstTask)).as("First iter should not be closed on its last element").isFalse(); + assertThat(reader.isIteratorClosed(firstTask)) + .as("First iter should not be closed on its last element") + .isFalse(); assertThat(reader.next()).isTrue(); - assertThat(reader.isIteratorClosed(firstTask)).as("First iter should be closed after moving to second iter").isTrue(); - assertThat(reader.isIteratorClosed(secondTask)).as("Second iter should not be closed on its last element").isFalse(); + assertThat(reader.isIteratorClosed(firstTask)) + .as("First iter should be closed after moving to second iter") + .isTrue(); + assertThat(reader.isIteratorClosed(secondTask)) + .as("Second iter should not be closed on its last element") + .isFalse(); assertThat(reader.next()).isFalse(); assertThat(reader.isIteratorClosed(firstTask)).isTrue(); @@ -175,7 +184,9 @@ public void testClosureWithoutAnyRead() throws IOException { tasks.forEach( t -> - assertThat(reader.hasIterator(t)).as("Iterator should not be created eagerly for tasks").isFalse()); + assertThat(reader.hasIterator(t)) + .as("Iterator should not be created eagerly for tasks") + .isFalse()); } @Test @@ -199,7 +210,9 @@ public void testExplicitClosure() throws IOException { tasks.forEach( t -> { if (reader.hasIterator(t)) { - assertThat(reader.isIteratorClosed(t)).as("Iterator should be closed after read exhausion").isTrue(); + assertThat(reader.isIteratorClosed(t)) + .as("Iterator should be closed after read exhausion") + .isTrue(); } }); } @@ -221,10 +234,14 @@ public void testIdempotentExplicitClosure() throws IOException { for (int closeAttempt = 0; closeAttempt < 5; closeAttempt++) { reader.close(); for (int i = 0; i < 5; i++) { - assertThat(reader.isIteratorClosed(tasks.get(i))).as("Iterator should be closed after read exhausion").isTrue(); + assertThat(reader.isIteratorClosed(tasks.get(i))) + .as("Iterator should be closed after read exhausion") + .isTrue(); } for (int i = 5; i < 10; i++) { - assertThat(reader.hasIterator(tasks.get(i))).as("Iterator should not be created eagerly for tasks").isFalse(); + assertThat(reader.hasIterator(tasks.get(i))) + .as("Iterator should not be created eagerly for tasks") + .isFalse(); } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java index 080ba7d59359..934926e17740 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java @@ -64,8 +64,7 @@ public class TestChangelogReader extends TestBase { private DataFile dataFile1; private DataFile dataFile2; - @TempDir - private Path temp; + @TempDir private Path temp; @BeforeEach public void before() throws IOException { @@ -255,8 +254,6 @@ private Object[] toJava(InternalRow row) { private DataFile writeDataFile(List records) throws IOException { // records all use IDs that are in bucket id_bucket=0 return FileHelpers.writeDataFile( - table, - Files.localOutput(temp.toFile()), - TestHelpers.Row.of(0), records); + table, Files.localOutput(temp.toFile()), TestHelpers.Row.of(0), records); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java index b583acf443e3..4959cd2a9e06 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; + import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -102,8 +103,7 @@ public void testMergeSchemaWithoutAcceptAnySchema() throws Exception { "{ \"id\": 3, \"data\": \"c\", \"new_col\": 12.06 }", "{ \"id\": 4, \"data\": \"d\", \"new_col\": 14.41 }"); - assertThatThrownBy( - () -> threeColDF.writeTo(tableName).option("merge-schema", "true").append()) + assertThatThrownBy(() -> threeColDF.writeTo(tableName).option("merge-schema", "true").append()) .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Cannot write to `testhadoop`.`default`.`table`, the reason is too many data columns"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 12b6347282c6..917e2035bc65 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -70,8 +70,7 @@ public class TestDataSourceOptions extends TestBaseWithCatalog { optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); private static SparkSession spark = null; - @TempDir - private Path temp; + @TempDir private Path temp; @BeforeAll public static void startSpark() { @@ -220,7 +219,9 @@ public void testSplitOptionsOverridesTableProperties() throws IOException { .option(SparkReadOptions.SPLIT_SIZE, String.valueOf(splitSize)) .load(tableLocation); - assertThat(2).as("Spark partitions should match").isEqualTo(resultDf.javaRDD().getNumPartitions()); + assertThat(2) + .as("Spark partitions should match") + .isEqualTo(resultDf.javaRDD().getNumPartitions()); } @Test diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 314237f62a09..84c99a575c8d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -87,8 +87,7 @@ public class TestForwardCompatibility { .addField("identity", 1, "id_zero") .build(); - @TempDir - private Path temp; + @TempDir private Path temp; private static SparkSession spark = null; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 0cebf6c272d9..0b0b8af995e3 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -117,8 +117,7 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("id").build(); - @TempDir - private Path temp; + @TempDir private Path temp; public abstract Table createTable( TableIdentifier ident, Schema schema, PartitionSpec spec, Map properties); @@ -436,13 +435,19 @@ public void testCountEntriesTable() { // count entries assertThat(expectedEntryCount) - .as("Count should return " + expectedEntryCount) - .isEqualTo(spark.read().format("iceberg").load(loadLocation(tableIdentifier, "entries")).count()); + .as("Count should return " + expectedEntryCount) + .isEqualTo( + spark.read().format("iceberg").load(loadLocation(tableIdentifier, "entries")).count()); // count all_entries assertThat(expectedEntryCount) - .as("Count should return " + expectedEntryCount) - .isEqualTo(spark.read().format("iceberg").load(loadLocation(tableIdentifier, "all_entries")).count()); + .as("Count should return " + expectedEntryCount) + .isEqualTo( + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "all_entries")) + .count()); } @Test @@ -565,7 +570,7 @@ public void testV1EntriesTableWithSnapshotIdInheritance() throws Exception { String.format( "CREATE TABLE parquet_table (data string, id int) " + "USING parquet PARTITIONED BY (id) LOCATION '%s'", - temp.toFile())); + temp.toFile())); List records = Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); @@ -592,7 +597,6 @@ public void testV1EntriesTableWithSnapshotIdInheritance() throws Exception { long snapshotId = table.currentSnapshot().snapshotId(); - assertThat(actual).as("Entries table should have 2 rows").hasSize(2); assertThat(0).as("Sequence number must match").isEqualTo(actual.get(0).getLong(0)); assertThat(snapshotId).as("Snapshot id must match").isEqualTo(actual.get(0).getLong(1)); @@ -1290,8 +1294,8 @@ public void testUnpartitionedPartitionsTable() { Table partitionsTable = loadTable(tableIdentifier, "partitions"); assertThat(expectedSchema) - .as("Schema should not have partition field") - .isEqualTo(partitionsTable.schema().asStruct()); + .as("Schema should not have partition field") + .isEqualTo(partitionsTable.schema().asStruct()); GenericRecordBuilder builder = new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); @@ -1482,10 +1486,10 @@ public void testPartitionsTableLastUpdatedSnapshot() { RewriteManifests.Result rewriteManifestResult = SparkActions.get().rewriteManifests(table).execute(); assertThat(rewriteManifestResult.rewrittenManifests()) - .as("rewrite replaced 2 manifests").hasSize(2); + .as("rewrite replaced 2 manifests") + .hasSize(2); - assertThat(rewriteManifestResult.addedManifests()) - .as("rewrite added 1 manifests").hasSize(1); + assertThat(rewriteManifestResult.addedManifests()).as("rewrite added 1 manifests").hasSize(1); List actual = spark @@ -1734,8 +1738,8 @@ public synchronized void testSnapshotReadAfterAddColumn() { Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); assertThat(originalRecords) - .as("Records should match") - .isEqualTo(resultDf.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf.orderBy("id").collectAsList()); Snapshot snapshotBeforeAddColumn = table.currentSnapshot(); @@ -1765,8 +1769,8 @@ public synchronized void testSnapshotReadAfterAddColumn() { Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); assertThat(updatedRecords) - .as("Records should match") - .isEqualTo(resultDf2.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf2.orderBy("id").collectAsList()); Dataset resultDf3 = spark @@ -1776,12 +1780,10 @@ public synchronized void testSnapshotReadAfterAddColumn() { .load(loadLocation(tableIdentifier)); assertThat(originalRecords) - .as("Records should match") - .isEqualTo(resultDf3.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf3.orderBy("id").collectAsList()); - assertThat(originalSparkSchema) - .as("Schemas should match") - .isEqualTo(resultDf3.schema()); + assertThat(originalSparkSchema).as("Schemas should match").isEqualTo(resultDf3.schema()); } @Test @@ -1809,8 +1811,8 @@ public synchronized void testSnapshotReadAfterDropColumn() { Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); assertThat(originalRecords) - .as("Records should match") - .isEqualTo(resultDf.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf.orderBy("id").collectAsList()); long tsBeforeDropColumn = waitUntilAfter(System.currentTimeMillis()); table.updateSchema().deleteColumn("data").commit(); @@ -1839,8 +1841,8 @@ public synchronized void testSnapshotReadAfterDropColumn() { Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); assertThat(updatedRecords) - .as("Records should match") - .isEqualTo(resultDf2.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf2.orderBy("id").collectAsList()); Dataset resultDf3 = spark @@ -1850,12 +1852,10 @@ public synchronized void testSnapshotReadAfterDropColumn() { .load(loadLocation(tableIdentifier)); assertThat(originalRecords) - .as("Records should match") - .isEqualTo(resultDf3.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf3.orderBy("id").collectAsList()); - assertThat(originalSparkSchema) - .as("Schemas should match") - .isEqualTo(resultDf3.schema()); + assertThat(originalSparkSchema).as("Schemas should match").isEqualTo(resultDf3.schema()); // At tsAfterDropColumn, there has been a schema change, but no new snapshot, // so the snapshot as of tsAfterDropColumn is the same as that as of tsBeforeDropColumn. @@ -1867,12 +1867,10 @@ public synchronized void testSnapshotReadAfterDropColumn() { .load(loadLocation(tableIdentifier)); assertThat(originalRecords) - .as("Records should match") - .isEqualTo(resultDf4.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf4.orderBy("id").collectAsList()); - assertThat(originalSparkSchema) - .as("Schemas should match") - .isEqualTo(resultDf4.schema()); + assertThat(originalSparkSchema).as("Schemas should match").isEqualTo(resultDf4.schema()); } @Test @@ -1898,8 +1896,8 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); assertThat(originalRecords) - .as("Records should match") - .isEqualTo(resultDf.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf.orderBy("id").collectAsList()); Snapshot snapshotBeforeAddColumn = table.currentSnapshot(); @@ -1930,8 +1928,8 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); assertThat(updatedRecords) - .as("Records should match") - .isEqualTo(resultDf2.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf2.orderBy("id").collectAsList()); table.updateSchema().deleteColumn("data").commit(); @@ -1946,8 +1944,8 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { Dataset resultDf3 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); assertThat(recordsAfterDropColumn) - .as("Records should match") - .isEqualTo(resultDf3.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf3.orderBy("id").collectAsList()); Dataset resultDf4 = spark @@ -1957,12 +1955,10 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { .load(loadLocation(tableIdentifier)); assertThat(originalRecords) - .as("Records should match") - .isEqualTo(resultDf4.orderBy("id").collectAsList()); + .as("Records should match") + .isEqualTo(resultDf4.orderBy("id").collectAsList()); - assertThat(originalSparkSchema) - .as("Schemas should match") - .isEqualTo(resultDf4.schema()); + assertThat(originalSparkSchema).as("Schemas should match").isEqualTo(resultDf4.schema()); } @Test @@ -1995,14 +1991,13 @@ public void testRemoveOrphanFilesActionSupport() throws InterruptedException { .execute(); assertThat(Iterables.isEmpty(result1.orphanFileLocations())) - .as("Should not delete any metadata files") - .isTrue(); + .as("Should not delete any metadata files") + .isTrue(); DeleteOrphanFiles.Result result2 = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); - assertThat(result2.orphanFileLocations()) - .as("Should delete 1 data file").hasSize(1); + assertThat(result2.orphanFileLocations()).as("Should delete 1 data file").hasSize(1); Dataset resultDF = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); List actualRecords = @@ -2048,7 +2043,9 @@ public void testFilesTablePartitionId() { .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) .collect(Collectors.toList()); - assertThat(ImmutableList.of(spec0, spec1)).as("Should have two partition specs").isEqualTo(actual); + assertThat(ImmutableList.of(spec0, spec1)) + .as("Should have two partition specs") + .isEqualTo(actual); } @Test @@ -2186,9 +2183,7 @@ public void testTableWithInt96Timestamp() throws IOException { .load(loadLocation(tableIdentifier)) .select("tmp_col") .collectAsList(); - assertThat(actual) - .as("Rows must match") - .containsExactlyInAnyOrderElementsOf(expected); + assertThat(actual).as("Rows must match").containsExactlyInAnyOrderElementsOf(expected); dropTable(tableIdentifier); } } @@ -2309,8 +2304,8 @@ private void assertDataFilePartitions( for (int i = 0; i < dataFiles.size(); ++i) { assertThat(expectedPartitionIds.get(i).intValue()) - .as("Data file should have partition of id " + expectedPartitionIds.get(i)) - .isEqualTo(dataFiles.get(i).partition().get(0, Integer.class).intValue()); + .as("Data file should have partition of id " + expectedPartitionIds.get(i)) + .isEqualTo(dataFiles.get(i).partition().get(0, Integer.class).intValue()); } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java index 9bf9b03e8f08..756c45ed03b4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; + import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -62,7 +63,7 @@ public void testRegisterIntegerBucketUDF() { assertThat(results).hasSize(1); assertThat((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -71,7 +72,7 @@ public void testRegisterShortBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_short_16(1S)").collectAsList(); assertThat(results).hasSize(1); assertThat((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -80,7 +81,7 @@ public void testRegisterByteBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_byte_16(1Y)").collectAsList(); assertThat(results).hasSize(1); assertThat((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -89,7 +90,7 @@ public void testRegisterLongBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_long_16(1L)").collectAsList(); assertThat(results).hasSize(1); assertThat((int) Transforms.bucket(16).bind(Types.LongType.get()).apply(1L)) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -98,7 +99,7 @@ public void testRegisterStringBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_string_16('hello')").collectAsList(); assertThat(results).hasSize(1); assertThat((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -107,7 +108,7 @@ public void testRegisterCharBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_char_16('hello')").collectAsList(); assertThat(results).hasSize(1); assertThat((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -116,7 +117,7 @@ public void testRegisterVarCharBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_varchar_16('hello')").collectAsList(); assertThat(results).hasSize(1); assertThat((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -125,11 +126,12 @@ public void testRegisterDateBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_date_16(DATE '2021-06-30')").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) - Transforms.bucket(16) + assertThat( + (int) + Transforms.bucket(16) .bind(Types.DateType.get()) .apply(DateTimeUtils.fromJavaDate(Date.valueOf("2021-06-30")))) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -141,12 +143,14 @@ public void testRegisterTimestampBucketUDF() { .sql("SELECT iceberg_bucket_timestamp_16(TIMESTAMP '2021-06-30 00:00:00.000')") .collectAsList(); assertThat(results).hasSize(1); - assertThat((int) - Transforms.bucket(16) + assertThat( + (int) + Transforms.bucket(16) .bind(Types.TimestampType.withZone()) .apply( - DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2021-06-30 00:00:00.000")))) - .isEqualTo(results.get(0).getInt(0)); + DateTimeUtils.fromJavaTimestamp( + Timestamp.valueOf("2021-06-30 00:00:00.000")))) + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -154,11 +158,12 @@ public void testRegisterBinaryBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_binary_16", DataTypes.BinaryType, 16); List results = spark.sql("SELECT iceberg_bucket_binary_16(X'0020001F')").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) - Transforms.bucket(16) + assertThat( + (int) + Transforms.bucket(16) .bind(Types.BinaryType.get()) .apply(ByteBuffer.wrap(new byte[] {0x00, 0x20, 0x00, 0x1F}))) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -166,8 +171,12 @@ public void testRegisterDecimalBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_decimal_16", new DecimalType(4, 2), 16); List results = spark.sql("SELECT iceberg_bucket_decimal_16(11.11)").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) Transforms.bucket(16).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11"))) - .isEqualTo(results.get(0).getInt(0)); + assertThat( + (int) + Transforms.bucket(16) + .bind(Types.DecimalType.of(4, 2)) + .apply(new BigDecimal("11.11"))) + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -206,7 +215,7 @@ public void testRegisterIntegerTruncateUDF() { List results = spark.sql("SELECT iceberg_truncate_int_4(1)").collectAsList(); assertThat(results).hasSize(1); assertThat(Transforms.truncate(4).bind(Types.IntegerType.get()).apply(1)) - .isEqualTo(results.get(0).getInt(0)); + .isEqualTo(results.get(0).getInt(0)); } @Test @@ -215,7 +224,7 @@ public void testRegisterLongTruncateUDF() { List results = spark.sql("SELECT iceberg_truncate_long_4(1L)").collectAsList(); assertThat(results).hasSize(1); assertThat(Transforms.truncate(4).bind(Types.LongType.get()).apply(1L)) - .isEqualTo(results.get(0).getLong(0)); + .isEqualTo(results.get(0).getLong(0)); } @Test @@ -223,7 +232,9 @@ public void testRegisterDecimalTruncateUDF() { IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_decimal_4", new DecimalType(4, 2), 4); List results = spark.sql("SELECT iceberg_truncate_decimal_4(11.11)").collectAsList(); assertThat(results).hasSize(1); - assertThat(Transforms.truncate(4).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11"))).isEqualTo(results.get(0).getDecimal(0)); + assertThat( + Transforms.truncate(4).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11"))) + .isEqualTo(results.get(0).getDecimal(0)); } @Test @@ -232,6 +243,6 @@ public void testRegisterStringTruncateUDF() { List results = spark.sql("SELECT iceberg_truncate_string_4('hello')").collectAsList(); assertThat(results).hasSize(1); assertThat(Transforms.truncate(4).bind(Types.StringType.get()).apply("hello")) - .isEqualTo(results.get(0).getString(0)); + .isEqualTo(results.get(0).getString(0)); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java index bc945209da87..0696c3291e36 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java @@ -53,8 +53,7 @@ public class TestMetadataTableReadableMetrics extends TestBaseWithCatalog { - @TempDir - private Path temp; + @TempDir private Path temp; private static final Types.StructType LEAF_STRUCT_TYPE = Types.StructType.of( @@ -125,8 +124,7 @@ private Table createPrimitiveTable() throws IOException { createPrimitiveRecord( false, 2, 2L, Float.NaN, 2.0D, new BigDecimal("2.00"), "2", null, null)); - DataFile dataFile = - FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); table.newAppend().appendFile(dataFile).commit(); return table; } @@ -144,8 +142,7 @@ private Pair createNestedTable() throws IOException { createNestedRecord(0L, 0.0), createNestedRecord(1L, Double.NaN), createNestedRecord(null, null)); - DataFile dataFile = - FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); table.newAppend().appendFile(dataFile).commit(); return Pair.of(table, dataFile); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java index 01376890df6a..f0e5b9e2045e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java @@ -48,8 +48,7 @@ public class TestPathIdentifier extends TestBase { new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - @TempDir - private Path temp; + @TempDir private Path temp; private File tableLocation; private PathIdentifier identifier; private SparkCatalog sparkCatalog; @@ -77,8 +76,7 @@ public void testPathIdentifier() throws TableAlreadyExistsException, NoSuchTable assertThat(table.table().location()).isEqualTo(tableLocation.getAbsolutePath()); assertThat(table.table()).isInstanceOf(BaseTable.class); - assertThat(((BaseTable) table.table()).operations()) - .isInstanceOf(HadoopTableOperations.class); + assertThat(((BaseTable) table.table()).operations()).isInstanceOf(HadoopTableOperations.class); assertThat(sparkCatalog.loadTable(identifier)).isEqualTo(table); assertThat(sparkCatalog.dropTable(identifier)).isTrue(); 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 44aafb9bf2fa..317933805d64 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 @@ -50,8 +50,7 @@ public abstract class TestReadProjection { protected abstract Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; - @TempDir - private Path temp; + @TempDir private Path temp; @Test public void testFullProjection() throws Exception { @@ -66,13 +65,13 @@ public void testFullProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, schema, record); - assertThat(34L).as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + assertThat(34L) + .as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); - assertThat(0).as("Should contain the correct data value") - .isEqualTo(cmp); + assertThat(0).as("Should contain the correct data value").isEqualTo(cmp); } @Test @@ -97,9 +96,10 @@ public void testReorderedFullProjection() throws Exception { Record projected = writeAndRead("reordered_full_projection", schema, reordered, record); - assertThat("test").as("Should contain the correct 0 value").isEqualTo(projected.get(0).toString()); + assertThat("test") + .as("Should contain the correct 0 value") + .isEqualTo(projected.get(0).toString()); assertThat(34L).as("Should contain the correct 1 value").isEqualTo(projected.get(1)); - } @Test @@ -126,7 +126,9 @@ public void testReorderedProjection() throws Exception { Record projected = writeAndRead("reordered_projection", schema, reordered, record); assertThat(projected.get(0)).as("Should contain the correct 0 value").isNull(); - assertThat("test").as("Should contain the correct 1 value").isEqualTo(projected.get(1).toString()); + assertThat("test") + .as("Should contain the correct 1 value") + .isEqualTo(projected.get(1).toString()); assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); } @@ -145,8 +147,7 @@ public void testEmptyProjection() throws Exception { assertThat(projected).as("Should read a non-null record").isNotNull(); // this is expected because there are no values - assertThatThrownBy(() -> projected.get(0)) - .isInstanceOf(ArrayIndexOutOfBoundsException.class); + assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); } @Test @@ -164,7 +165,9 @@ public void testBasicProjection() throws Exception { Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); assertThat(projected.getField("data")).as("Should not project data").isNull(); - assertThat(34L).as("Should contain the correct id value").isEqualTo((long) projected.getField("id")); + assertThat(34L) + .as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); @@ -193,7 +196,9 @@ public void testRename() throws Exception { Types.NestedField.optional(1, "renamed", Types.StringType.get())); Record projected = writeAndRead("project_and_rename", writeSchema, readSchema, record); - assertThat(34L).as("Should contain the correct id value").isEqualTo((long) projected.getField("id")); + assertThat(34L) + .as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("renamed")); @@ -223,7 +228,9 @@ public void testNestedStructProjection() throws Exception { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Record projectedLocation = (Record) projected.getField("location"); - assertThat(34L).as("Should contain the correct id value").isEqualTo((long) projected.getField("id")); + assertThat(34L) + .as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); assertThat(projectedLocation).as("Should not project location").isNull(); Schema latOnly = @@ -239,8 +246,8 @@ public void testNestedStructProjection() throws Exception { assertThat(projected.getField("location")).as("Should project location").isNotNull(); assertThat(projectedLocation.getField("long")).as("Should not project longitude").isNull(); assertThat(52.995143f) - .as("Should project latitude") - .isCloseTo((float) projectedLocation.getField("lat"), within(0.000001f)); + .as("Should project latitude") + .isCloseTo((float) projectedLocation.getField("lat"), within(0.000001f)); Schema longOnly = new Schema( @@ -256,8 +263,8 @@ public void testNestedStructProjection() throws Exception { assertThat(projected.getField("location")).as("Should project location").isNotNull(); assertThat(projectedLocation.getField("lat")).as("Should not project latitude").isNull(); assertThat(-1.539054f) - .as("Should project longitude") - .isCloseTo((float) projectedLocation.getField("long"), within(0.000001f)); + .as("Should project longitude") + .isCloseTo((float) projectedLocation.getField("long"), within(0.000001f)); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); @@ -266,11 +273,11 @@ public void testNestedStructProjection() throws Exception { assertThat(projected.getField("id")).as("Should not project id").isNull(); assertThat(projected.getField("location")).as("Should project location").isNotNull(); assertThat(52.995143f) - .as("Should project latitude") - .isCloseTo((float) projectedLocation.getField("lat"), within(0.000001f)); + .as("Should project latitude") + .isCloseTo((float) projectedLocation.getField("lat"), within(0.000001f)); assertThat(-1.539054f) - .as("Should project longitude") - .isCloseTo((float) projectedLocation.getField("long"), within(0.000001f)); + .as("Should project longitude") + .isCloseTo((float) projectedLocation.getField("long"), within(0.000001f)); } @Test @@ -292,26 +299,31 @@ public void testMapProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat(34L).as("Should contain the correct id value").isEqualTo((long) projected.getField("id")); + assertThat(34L) + .as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); assertThat(projected.getField("properties")).as("Should not project properties map").isNull(); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(properties).as("Should project entire map") - .isEqualTo(toStringMap((Map) projected.getField("properties"))); + assertThat(properties) + .as("Should project entire map") + .isEqualTo(toStringMap((Map) projected.getField("properties"))); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(properties).as("Should project entire map") - .isEqualTo(toStringMap((Map) projected.getField("properties"))); + assertThat(properties) + .as("Should project entire map") + .isEqualTo(toStringMap((Map) projected.getField("properties"))); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(properties).as("Should project entire map") - .isEqualTo(toStringMap((Map) projected.getField("properties"))); + assertThat(properties) + .as("Should project entire map") + .isEqualTo(toStringMap((Map) projected.getField("properties"))); } private Map toStringMap(Map map) { @@ -355,35 +367,38 @@ public void testMapOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat(34L).as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + assertThat(34L) + .as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); assertThat(projected.getField("locations")).as("Should not project locations map").isNull(); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(record.getField("locations")).as("Should project locations map") - .isEqualTo(toStringMap((Map) projected.getField("locations"))); + assertThat(record.getField("locations")) + .as("Should project locations map") + .isEqualTo(toStringMap((Map) projected.getField("locations"))); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); assertThat(projected.getField("id")).as("Should not project id").isNull(); Map locations = toStringMap((Map) projected.getField("locations")); assertThat(locations).as("Should project locations map").isNotNull(); - assertThat(Sets.newHashSet("L1", "L2")).as("Should contain L1 and L2") - .isEqualTo(locations.keySet()); + assertThat(Sets.newHashSet("L1", "L2")) + .as("Should contain L1 and L2") + .isEqualTo(locations.keySet()); Record projectedL1 = (Record) locations.get("L1"); assertThat(projectedL1).as("L1 should not be null").isNotNull(); assertThat(53.992811f) - .as("L1 should contain lat") - .isCloseTo((float) projectedL1.getField("lat"), within(0.000001f)); + .as("L1 should contain lat") + .isCloseTo((float) projectedL1.getField("lat"), within(0.000001f)); assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); Record projectedL2 = (Record) locations.get("L2"); assertThat(projectedL2).as("L2 should not be null").isNotNull(); assertThat(52.995143f) - .as("L2 should contain lat") - .isCloseTo((float) projectedL2.getField("lat"), within(0.000001f)); + .as("L2 should contain lat") + .isCloseTo((float) projectedL2.getField("lat"), within(0.000001f)); assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); projected = @@ -392,22 +407,23 @@ public void testMapOfStructsProjection() throws IOException { locations = toStringMap((Map) projected.getField("locations")); assertThat(locations).as("Should project locations map").isNotNull(); - assertThat(Sets.newHashSet("L1", "L2")).as("Should contain L1 and L2") - .isEqualTo(locations.keySet()); + assertThat(Sets.newHashSet("L1", "L2")) + .as("Should contain L1 and L2") + .isEqualTo(locations.keySet()); projectedL1 = (Record) locations.get("L1"); assertThat(projectedL1).as("L1 should not be null").isNotNull(); assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); assertThat(-1.542616f) - .as("L1 should contain long") - .isCloseTo((float) projectedL1.getField("long"), within(0.000001f)); + .as("L1 should contain long") + .isCloseTo((float) projectedL1.getField("long"), within(0.000001f)); projectedL2 = (Record) locations.get("L2"); assertThat(projectedL2).as("L2 should not be null").isNotNull(); assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); assertThat(-1.539054f) - .as("L2 should contain long") - .isCloseTo((float) projectedL2.getField("long"), within(0.000001f)); + .as("L2 should contain long") + .isCloseTo((float) projectedL2.getField("long"), within(0.000001f)); Schema latitiudeRenamed = new Schema( @@ -425,22 +441,23 @@ public void testMapOfStructsProjection() throws IOException { assertThat(projected.getField("id")).as("Should not project id").isNull(); locations = toStringMap((Map) projected.getField("locations")); assertThat(locations).as("Should project locations map").isNotNull(); - assertThat(Sets.newHashSet("L1", "L2")).as("Should contain L1 and L2") - .isEqualTo(locations.keySet()); + assertThat(Sets.newHashSet("L1", "L2")) + .as("Should contain L1 and L2") + .isEqualTo(locations.keySet()); projectedL1 = (Record) locations.get("L1"); assertThat(projectedL1).as("L1 should not be null").isNotNull(); assertThat(53.992811f) - .as("L1 should contain latitude") - .isCloseTo((float) projectedL1.getField("latitude"), within(0.000001f)); + .as("L1 should contain latitude") + .isCloseTo((float) projectedL1.getField("latitude"), within(0.000001f)); assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); projectedL2 = (Record) locations.get("L2"); assertThat(projectedL2).as("L2 should not be null").isNotNull(); assertThat(52.995143f) - .as("L2 should contain latitude") - .isCloseTo((float) projectedL2.getField("latitude"), within(0.000001f)); + .as("L2 should contain latitude") + .isCloseTo((float) projectedL2.getField("latitude"), within(0.000001f)); assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); } @@ -462,8 +479,9 @@ public void testListProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat(34L).as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + assertThat(34L) + .as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); assertThat(projected.getField("values")).as("Should not project values list").isNull(); Schema elementOnly = writeSchema.select("values.element"); @@ -505,14 +523,16 @@ public void testListOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat(34L).as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + assertThat(34L) + .as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); assertThat(projected.getField("points")).as("Should not project points list").isNull(); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(record.getField("points")).as("Should project points list") - .isEqualTo(projected.getField("points")); + assertThat(record.getField("points")) + .as("Should project points list") + .isEqualTo(projected.getField("points")); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); assertThat(projected.getField("id")).as("Should not project id").isNull(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java index 2846333234e4..8d4dd9881ecb 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; + import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; @@ -50,17 +51,23 @@ public void testAggregates() { Max max = new Max(namedReference); Expression expectedMax = Expressions.max(unquoted); Expression actualMax = SparkAggregates.convert(max); - assertThat(expectedMax.toString()).as("Max must match").isEqualTo(String.valueOf(actualMax)); + assertThat(expectedMax.toString()) + .as("Max must match") + .isEqualTo(String.valueOf(actualMax)); Min min = new Min(namedReference); Expression expectedMin = Expressions.min(unquoted); Expression actualMin = SparkAggregates.convert(min); - assertThat(expectedMin.toString()).as("Min must match").isEqualTo(String.valueOf(actualMin)); + assertThat(expectedMin.toString()) + .as("Min must match") + .isEqualTo(String.valueOf(actualMin)); Count count = new Count(namedReference, false); Expression expectedCount = Expressions.count(unquoted); Expression actualCount = SparkAggregates.convert(count); - assertThat(expectedCount.toString()).as("Count must match").isEqualTo(String.valueOf(actualCount)); + assertThat(expectedCount.toString()) + .as("Count must match") + .isEqualTo(String.valueOf(actualCount)); Count countDistinct = new Count(namedReference, true); Expression convertedCountDistinct = SparkAggregates.convert(countDistinct); @@ -69,7 +76,9 @@ public void testAggregates() { CountStar countStar = new CountStar(); Expression expectedCountStar = Expressions.countStar(); Expression actualCountStar = SparkAggregates.convert(countStar); - assertThat(expectedCountStar.toString()).as("CountStar must match").isEqualTo(String.valueOf(actualCountStar)); + assertThat(expectedCountStar.toString()) + .as("CountStar must match") + .isEqualTo(String.valueOf(actualCountStar)); }); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java index 7911ce0d966b..9ff3de74f6fb 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; + import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; @@ -121,9 +122,7 @@ public void testCacheEnabledAndExpirationDisabled() { .isInstanceOfSatisfying( CachingCatalog.class, icebergCatalog -> { - assertThat(icebergCatalog) - .extracting("expirationIntervalMillis") - .isEqualTo(-1L); + assertThat(icebergCatalog).extracting("expirationIntervalMillis").isEqualTo(-1L); }); } @@ -136,8 +135,7 @@ public void testCacheDisabledImplicitly() { .extracting("icebergCatalog") .isInstanceOfSatisfying( Catalog.class, - icebergCatalog -> - assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); + icebergCatalog -> assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); } private SparkSessionCatalog sparkSessionCatalog() { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index 23949e114675..f0a1a28509e1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -133,8 +133,7 @@ public static void stopSpark() { currentSpark.stop(); } - @TempDir - private Path temp; + @TempDir private Path temp; private String tableLocation = null; @BeforeEach diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java index b8830b611e4e..4a0f9b58d8e9 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; + import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; @@ -110,7 +111,9 @@ public void testTaskSetPlanning() throws NoSuchTableException, IOException { .option(SparkReadOptions.SCAN_TASK_SET_ID, setID) .option(SparkReadOptions.SPLIT_SIZE, tasks.get(0).file().fileSizeInBytes()) .load(tableName); - assertThat(2).as("Num partitions should match").isEqualTo(scanDF.javaRDD().getNumPartitions()); + assertThat(2) + .as("Num partitions should match") + .isEqualTo(scanDF.javaRDD().getNumPartitions()); // load the staged file set and make sure we combine both files into a single split scanDF = @@ -120,7 +123,9 @@ public void testTaskSetPlanning() throws NoSuchTableException, IOException { .option(SparkReadOptions.SCAN_TASK_SET_ID, setID) .option(SparkReadOptions.SPLIT_SIZE, Long.MAX_VALUE) .load(tableName); - assertThat(1).as("Num partitions should match").isEqualTo(scanDF.javaRDD().getNumPartitions()); + assertThat(1) + .as("Num partitions should match") + .isEqualTo(scanDF.javaRDD().getNumPartitions()); } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java index 621ac0de2df2..21f4cc4120fd 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; + import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java index df6308d7ce8b..7f7de2172692 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java @@ -17,6 +17,7 @@ * under the License. */ package org.apache.iceberg.spark.source; + import static org.assertj.core.api.Assertions.assertThat; import com.fasterxml.jackson.databind.node.ObjectNode; @@ -35,8 +36,10 @@ public void testJsonConversion() { new StreamingOffset(System.currentTimeMillis(), 3L, false), new StreamingOffset(System.currentTimeMillis(), 4L, true) }; - assertThat(expected).as("StreamingOffsets should match") - .isEqualTo(Arrays.stream(expected).map(elem -> StreamingOffset.fromJson(elem.json())).toArray()); + assertThat(expected) + .as("StreamingOffsets should match") + .isEqualTo( + Arrays.stream(expected).map(elem -> StreamingOffset.fromJson(elem.json())).toArray()); } @Test diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index 6d89a0e589f4..263e7c59bb20 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -57,8 +57,7 @@ public class TestStructuredStreaming { optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); private static SparkSession spark = null; - @TempDir - private Path temp; + @TempDir private Path temp; @BeforeAll public static void startSpark() { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java index d1b09d090848..cf759a827e60 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java @@ -73,8 +73,7 @@ public class TestWriteMetricsConfig { required(4, "id", Types.IntegerType.get()), required(5, "data", Types.StringType.get())))); - @TempDir - private Path temp; + @TempDir private Path temp; private static SparkSession spark = null; private static JavaSparkContext sc = null; @@ -233,8 +232,7 @@ public void testBadCustomMetricCollectionForParquet() throws IOException { properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "counts"); properties.put("write.metadata.metrics.column.ids", "full"); - assertThatThrownBy( - () -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)) + assertThatThrownBy(() -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith( "Invalid metrics config, could not find column ids from table prop write.metadata.metrics.column.ids in schema table"); @@ -294,7 +292,7 @@ public void testCustomMetricCollectionForNestedParquet() throws IOException { Map upperBounds = file.upperBounds(); assertThat(upperBounds).hasSize(2); assertThat(upperBounds).containsKey(recordId.fieldId()); - + ByteBuffer recordDataUpperBound = upperBounds.get(recordData.fieldId()); assertThat(ByteBuffers.toByteArray(recordDataUpperBound)).hasSize(2); } From 3db3ae68eba1ca7acf10f48da500dd19535904aa Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Wed, 20 Dec 2023 13:48:56 +0530 Subject: [PATCH 05/15] fix assertThat to (actual, expected) --- .../spark/source/TestDataSourceOptions.java | 16 +-- .../source/TestIcebergSourceTablesBase.java | 74 +++++----- .../spark/source/TestIcebergSpark.java | 70 ++++----- .../spark/source/TestPathIdentifier.java | 4 +- .../spark/source/TestReadProjection.java | 134 +++++++++--------- .../spark/source/TestSparkAggregates.java | 16 +-- .../spark/source/TestSparkStagedScan.java | 8 +- .../spark/source/TestStreamingOffset.java | 7 +- .../spark/source/TestStructuredStreaming.java | 12 +- 9 files changed, 170 insertions(+), 171 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 917e2035bc65..5bcb33e0eb4a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -219,9 +219,9 @@ public void testSplitOptionsOverridesTableProperties() throws IOException { .option(SparkReadOptions.SPLIT_SIZE, String.valueOf(splitSize)) .load(tableLocation); - assertThat(2) + assertThat(resultDf.javaRDD().getNumPartitions()) .as("Spark partitions should match") - .isEqualTo(resultDf.javaRDD().getNumPartitions()); + .isEqualTo(2); } @Test @@ -299,7 +299,7 @@ public void testIncrementalScanOptions() throws IOException { .orderBy("id") .as(Encoders.bean(SimpleRecord.class)) .collectAsList(); - assertThat(expectedRecords.subList(1, 4)).as("Records should match").isEqualTo(result); + assertThat(result).as("Records should match").isEqualTo(expectedRecords.subList(1, 4)); // test (2nd snapshot, 3rd snapshot] incremental scan. Dataset resultDf = @@ -311,8 +311,8 @@ public void testIncrementalScanOptions() throws IOException { .load(tableLocation); List result1 = resultDf.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - assertThat(expectedRecords.subList(2, 3)).as("Records should match").isEqualTo(result1); - assertThat(1).as("Unprocessed count should match record count").isEqualTo(resultDf.count()); + assertThat(result1).as("Records should match").isEqualTo(expectedRecords.subList(2, 3)); + assertThat(resultDf.count()).as("Unprocessed count should match record count").isEqualTo(1); } @Test @@ -343,7 +343,7 @@ public void testMetadataSplitSizeOptionOverrideTableProperties() throws IOExcept .commit(); Dataset entriesDf = spark.read().format("iceberg").load(tableLocation + "#entries"); - assertThat(2).as("Num partitions must match").isEqualTo(entriesDf.javaRDD().getNumPartitions()); + assertThat(entriesDf.javaRDD().getNumPartitions()).as("Num partitions must match").isEqualTo(2); // override the table property using options entriesDf = @@ -352,7 +352,7 @@ public void testMetadataSplitSizeOptionOverrideTableProperties() throws IOExcept .format("iceberg") .option(SparkReadOptions.SPLIT_SIZE, String.valueOf(128 * 1024 * 1024)) .load(tableLocation + "#entries"); - assertThat(1).as("Num partitions must match").isEqualTo(entriesDf.javaRDD().getNumPartitions()); + assertThat(entriesDf.javaRDD().getNumPartitions()).as("Num partitions must match").isEqualTo(1); } @Test @@ -386,7 +386,7 @@ public void testDefaultMetadataSplitSize() throws IOException { Dataset metadataDf = spark.read().format("iceberg").load(tableLocation + "#entries"); int partitionNum = metadataDf.javaRDD().getNumPartitions(); - assertThat(expectedSplits).as("Spark partitions should match").isEqualTo(partitionNum); + assertThat(partitionNum).as("Spark partitions should match").isEqualTo(expectedSplits); } @Test diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 0b0b8af995e3..7fd1649df759 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -236,7 +236,7 @@ public void testEntriesTablePartitionedPrune() { .collectAsList(); assertThat(actual).as("Results should contain only one status").hasSize(1); - assertThat(1).as("That status should be Added (1)").isEqualTo(actual.get(0).getInt(0)); + assertThat(actual.get(0).getInt(0)).as("That status should be Added (1)").isEqualTo(1); } @Test @@ -434,20 +434,20 @@ public void testCountEntriesTable() { final int expectedEntryCount = 1; // count entries - assertThat(expectedEntryCount) + assertThat( + spark.read().format("iceberg").load(loadLocation(tableIdentifier, "entries")).count()) .as("Count should return " + expectedEntryCount) - .isEqualTo( - spark.read().format("iceberg").load(loadLocation(tableIdentifier, "entries")).count()); + .isEqualTo(expectedEntryCount); // count all_entries - assertThat(expectedEntryCount) - .as("Count should return " + expectedEntryCount) - .isEqualTo( + assertThat( spark .read() .format("iceberg") .load(loadLocation(tableIdentifier, "all_entries")) - .count()); + .count()) + .as("Count should return " + expectedEntryCount) + .isEqualTo(expectedEntryCount); } @Test @@ -598,10 +598,10 @@ public void testV1EntriesTableWithSnapshotIdInheritance() throws Exception { long snapshotId = table.currentSnapshot().snapshotId(); assertThat(actual).as("Entries table should have 2 rows").hasSize(2); - assertThat(0).as("Sequence number must match").isEqualTo(actual.get(0).getLong(0)); - assertThat(snapshotId).as("Snapshot id must match").isEqualTo(actual.get(0).getLong(1)); - assertThat(0).as("Sequence number must match").isEqualTo(actual.get(1).getLong(0)); - assertThat(snapshotId).as("Snapshot id must match").isEqualTo(actual.get(1).getLong(1)); + assertThat(actual.get(0).getLong(0)).as("Sequence number must match").isEqualTo(0); + assertThat(actual.get(0).getLong(1)).as("Snapshot id must match").isEqualTo(snapshotId); + assertThat(actual.get(1).getLong(0)).as("Sequence number must match").isEqualTo(0); + assertThat(actual.get(1).getLong(1)).as("Snapshot id must match").isEqualTo(snapshotId); } @Test @@ -1783,7 +1783,7 @@ public synchronized void testSnapshotReadAfterAddColumn() { .as("Records should match") .isEqualTo(resultDf3.orderBy("id").collectAsList()); - assertThat(originalSparkSchema).as("Schemas should match").isEqualTo(resultDf3.schema()); + assertThat(resultDf3.schema()).as("Schemas should match").isEqualTo(originalSparkSchema); } @Test @@ -1810,9 +1810,9 @@ public synchronized void testSnapshotReadAfterDropColumn() { Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - assertThat(originalRecords) + assertThat(resultDf.orderBy("id").collectAsList()) .as("Records should match") - .isEqualTo(resultDf.orderBy("id").collectAsList()); + .isEqualTo(originalRecords); long tsBeforeDropColumn = waitUntilAfter(System.currentTimeMillis()); table.updateSchema().deleteColumn("data").commit(); @@ -1840,9 +1840,9 @@ public synchronized void testSnapshotReadAfterDropColumn() { RowFactory.create(5, "C")); Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - assertThat(updatedRecords) + assertThat(resultDf2.orderBy("id").collectAsList()) .as("Records should match") - .isEqualTo(resultDf2.orderBy("id").collectAsList()); + .isEqualTo(updatedRecords); Dataset resultDf3 = spark @@ -1851,11 +1851,11 @@ public synchronized void testSnapshotReadAfterDropColumn() { .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) .load(loadLocation(tableIdentifier)); - assertThat(originalRecords) + assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") - .isEqualTo(resultDf3.orderBy("id").collectAsList()); + .isEqualTo(originalRecords); - assertThat(originalSparkSchema).as("Schemas should match").isEqualTo(resultDf3.schema()); + assertThat(resultDf3.schema()).as("Schemas should match").isEqualTo(originalSparkSchema); // At tsAfterDropColumn, there has been a schema change, but no new snapshot, // so the snapshot as of tsAfterDropColumn is the same as that as of tsBeforeDropColumn. @@ -1866,11 +1866,11 @@ public synchronized void testSnapshotReadAfterDropColumn() { .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) .load(loadLocation(tableIdentifier)); - assertThat(originalRecords) + assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") - .isEqualTo(resultDf4.orderBy("id").collectAsList()); + .isEqualTo(originalRecords); - assertThat(originalSparkSchema).as("Schemas should match").isEqualTo(resultDf4.schema()); + assertThat(resultDf4.schema()).as("Schemas should match").isEqualTo(originalSparkSchema); } @Test @@ -1895,9 +1895,9 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - assertThat(originalRecords) + assertThat(resultDf.orderBy("id").collectAsList()) .as("Records should match") - .isEqualTo(resultDf.orderBy("id").collectAsList()); + .isEqualTo(originalRecords); Snapshot snapshotBeforeAddColumn = table.currentSnapshot(); @@ -1927,9 +1927,9 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - assertThat(updatedRecords) + assertThat(resultDf2.orderBy("id").collectAsList()) .as("Records should match") - .isEqualTo(resultDf2.orderBy("id").collectAsList()); + .isEqualTo(updatedRecords); table.updateSchema().deleteColumn("data").commit(); @@ -1943,9 +1943,9 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { Dataset resultDf3 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); - assertThat(recordsAfterDropColumn) + assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") - .isEqualTo(resultDf3.orderBy("id").collectAsList()); + .isEqualTo(recordsAfterDropColumn); Dataset resultDf4 = spark @@ -1954,11 +1954,11 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); - assertThat(originalRecords) + assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") - .isEqualTo(resultDf4.orderBy("id").collectAsList()); + .isEqualTo(originalRecords); - assertThat(originalSparkSchema).as("Schemas should match").isEqualTo(resultDf4.schema()); + assertThat(resultDf4.schema()).as("Schemas should match").isEqualTo(originalSparkSchema); } @Test @@ -2003,7 +2003,7 @@ public void testRemoveOrphanFilesActionSupport() throws InterruptedException { List actualRecords = resultDF.as(Encoders.bean(SimpleRecord.class)).collectAsList(); - assertThat(records).as("Rows must match").isEqualTo(actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(records); } @Test @@ -2298,14 +2298,14 @@ private long totalSizeInBytes(Iterable dataFiles) { private void assertDataFilePartitions( List dataFiles, List expectedPartitionIds) { - assertThat(expectedPartitionIds) + assertThat(dataFiles) .as("Table should have " + expectedPartitionIds.size() + " data files") - .hasSameSizeAs(dataFiles); + .hasSameSizeAs(expectedPartitionIds); for (int i = 0; i < dataFiles.size(); ++i) { - assertThat(expectedPartitionIds.get(i).intValue()) + assertThat(dataFiles.get(i).partition().get(0, Integer.class).intValue()) .as("Data file should have partition of id " + expectedPartitionIds.get(i)) - .isEqualTo(dataFiles.get(i).partition().get(0, Integer.class).intValue()); + .isEqualTo(expectedPartitionIds.get(i).intValue()); } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java index 756c45ed03b4..e579712770e5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java @@ -62,8 +62,8 @@ public void testRegisterIntegerBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_int_16(1)").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)) - .isEqualTo(results.get(0).getInt(0)); + assertThat(results.get(0).getInt(0)) + .isEqualTo((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)); } @Test @@ -71,8 +71,8 @@ public void testRegisterShortBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_short_16", DataTypes.ShortType, 16); List results = spark.sql("SELECT iceberg_bucket_short_16(1S)").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)) - .isEqualTo(results.get(0).getInt(0)); + assertThat(results.get(0).getInt(0)) + .isEqualTo((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)); } @Test @@ -80,8 +80,8 @@ public void testRegisterByteBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_byte_16", DataTypes.ByteType, 16); List results = spark.sql("SELECT iceberg_bucket_byte_16(1Y)").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)) - .isEqualTo(results.get(0).getInt(0)); + assertThat(results.get(0).getInt(0)) + .isEqualTo((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)); } @Test @@ -89,8 +89,8 @@ public void testRegisterLongBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_long_16", DataTypes.LongType, 16); List results = spark.sql("SELECT iceberg_bucket_long_16(1L)").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) Transforms.bucket(16).bind(Types.LongType.get()).apply(1L)) - .isEqualTo(results.get(0).getInt(0)); + assertThat(results.get(0).getInt(0)) + .isEqualTo((int) Transforms.bucket(16).bind(Types.LongType.get()).apply(1L)); } @Test @@ -98,8 +98,8 @@ public void testRegisterStringBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_string_16", DataTypes.StringType, 16); List results = spark.sql("SELECT iceberg_bucket_string_16('hello')").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")) - .isEqualTo(results.get(0).getInt(0)); + assertThat(results.get(0).getInt(0)) + .isEqualTo((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")); } @Test @@ -107,8 +107,8 @@ public void testRegisterCharBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_char_16", new CharType(5), 16); List results = spark.sql("SELECT iceberg_bucket_char_16('hello')").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")) - .isEqualTo(results.get(0).getInt(0)); + assertThat(results.get(0).getInt(0)) + .isEqualTo((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")); } @Test @@ -116,8 +116,8 @@ public void testRegisterVarCharBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_varchar_16", new VarcharType(5), 16); List results = spark.sql("SELECT iceberg_bucket_varchar_16('hello')").collectAsList(); assertThat(results).hasSize(1); - assertThat((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")) - .isEqualTo(results.get(0).getInt(0)); + assertThat(results.get(0).getInt(0)) + .isEqualTo((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")); } @Test @@ -126,12 +126,12 @@ public void testRegisterDateBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_date_16(DATE '2021-06-30')").collectAsList(); assertThat(results).hasSize(1); - assertThat( + assertThat(results.get(0).getInt(0)) + .isEqualTo( (int) Transforms.bucket(16) .bind(Types.DateType.get()) - .apply(DateTimeUtils.fromJavaDate(Date.valueOf("2021-06-30")))) - .isEqualTo(results.get(0).getInt(0)); + .apply(DateTimeUtils.fromJavaDate(Date.valueOf("2021-06-30")))); } @Test @@ -143,14 +143,14 @@ public void testRegisterTimestampBucketUDF() { .sql("SELECT iceberg_bucket_timestamp_16(TIMESTAMP '2021-06-30 00:00:00.000')") .collectAsList(); assertThat(results).hasSize(1); - assertThat( + assertThat(results.get(0).getInt(0)) + .isEqualTo( (int) Transforms.bucket(16) .bind(Types.TimestampType.withZone()) .apply( DateTimeUtils.fromJavaTimestamp( - Timestamp.valueOf("2021-06-30 00:00:00.000")))) - .isEqualTo(results.get(0).getInt(0)); + Timestamp.valueOf("2021-06-30 00:00:00.000")))); } @Test @@ -158,12 +158,12 @@ public void testRegisterBinaryBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_binary_16", DataTypes.BinaryType, 16); List results = spark.sql("SELECT iceberg_bucket_binary_16(X'0020001F')").collectAsList(); assertThat(results).hasSize(1); - assertThat( + assertThat(results.get(0).getInt(0)) + .isEqualTo( (int) Transforms.bucket(16) .bind(Types.BinaryType.get()) - .apply(ByteBuffer.wrap(new byte[] {0x00, 0x20, 0x00, 0x1F}))) - .isEqualTo(results.get(0).getInt(0)); + .apply(ByteBuffer.wrap(new byte[] {0x00, 0x20, 0x00, 0x1F}))); } @Test @@ -171,12 +171,12 @@ public void testRegisterDecimalBucketUDF() { IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_decimal_16", new DecimalType(4, 2), 16); List results = spark.sql("SELECT iceberg_bucket_decimal_16(11.11)").collectAsList(); assertThat(results).hasSize(1); - assertThat( + assertThat(results.get(0).getInt(0)) + .isEqualTo( (int) Transforms.bucket(16) .bind(Types.DecimalType.of(4, 2)) - .apply(new BigDecimal("11.11"))) - .isEqualTo(results.get(0).getInt(0)); + .apply(new BigDecimal("11.11"))); } @Test @@ -214,8 +214,8 @@ public void testRegisterIntegerTruncateUDF() { IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_int_4", DataTypes.IntegerType, 4); List results = spark.sql("SELECT iceberg_truncate_int_4(1)").collectAsList(); assertThat(results).hasSize(1); - assertThat(Transforms.truncate(4).bind(Types.IntegerType.get()).apply(1)) - .isEqualTo(results.get(0).getInt(0)); + assertThat(results.get(0).getInt(0)) + .isEqualTo(Transforms.truncate(4).bind(Types.IntegerType.get()).apply(1)); } @Test @@ -223,8 +223,8 @@ public void testRegisterLongTruncateUDF() { IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_long_4", DataTypes.LongType, 4); List results = spark.sql("SELECT iceberg_truncate_long_4(1L)").collectAsList(); assertThat(results).hasSize(1); - assertThat(Transforms.truncate(4).bind(Types.LongType.get()).apply(1L)) - .isEqualTo(results.get(0).getLong(0)); + assertThat(results.get(0).getLong(0)) + .isEqualTo(Transforms.truncate(4).bind(Types.LongType.get()).apply(1L)); } @Test @@ -232,9 +232,9 @@ public void testRegisterDecimalTruncateUDF() { IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_decimal_4", new DecimalType(4, 2), 4); List results = spark.sql("SELECT iceberg_truncate_decimal_4(11.11)").collectAsList(); assertThat(results).hasSize(1); - assertThat( - Transforms.truncate(4).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11"))) - .isEqualTo(results.get(0).getDecimal(0)); + assertThat(results.get(0).getDecimal(0)) + .isEqualTo( + Transforms.truncate(4).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11"))); } @Test @@ -242,7 +242,7 @@ public void testRegisterStringTruncateUDF() { IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_string_4", DataTypes.StringType, 4); List results = spark.sql("SELECT iceberg_truncate_string_4('hello')").collectAsList(); assertThat(results).hasSize(1); - assertThat(Transforms.truncate(4).bind(Types.StringType.get()).apply("hello")) - .isEqualTo(results.get(0).getString(0)); + assertThat(results.get(0).getString(0)) + .isEqualTo(Transforms.truncate(4).bind(Types.StringType.get()).apply("hello")); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java index f0e5b9e2045e..bb026b2ab2da 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java @@ -74,11 +74,11 @@ public void testPathIdentifier() throws TableAlreadyExistsException, NoSuchTable sparkCatalog.createTable( identifier, SparkSchemaUtil.convert(SCHEMA), new Transform[0], ImmutableMap.of()); - assertThat(table.table().location()).isEqualTo(tableLocation.getAbsolutePath()); + assertThat(tableLocation.getAbsolutePath()).isEqualTo(table.table().location()); assertThat(table.table()).isInstanceOf(BaseTable.class); assertThat(((BaseTable) table.table()).operations()).isInstanceOf(HadoopTableOperations.class); - assertThat(sparkCatalog.loadTable(identifier)).isEqualTo(table); + assertThat(table).isEqualTo(sparkCatalog.loadTable(identifier)); assertThat(sparkCatalog.dropTable(identifier)).isTrue(); } } 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 317933805d64..ce0edba8d516 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 @@ -65,13 +65,13 @@ public void testFullProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, schema, record); - assertThat(34L) + assertThat((long) projected.getField("id")) .as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + .isEqualTo(34L); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); - assertThat(0).as("Should contain the correct data value").isEqualTo(cmp); + assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); } @Test @@ -96,10 +96,10 @@ public void testReorderedFullProjection() throws Exception { Record projected = writeAndRead("reordered_full_projection", schema, reordered, record); - assertThat("test") + assertThat(projected.get(0).toString()) .as("Should contain the correct 0 value") - .isEqualTo(projected.get(0).toString()); - assertThat(34L).as("Should contain the correct 1 value").isEqualTo(projected.get(1)); + .isEqualTo("test"); + assertThat(projected.get(1)).as("Should contain the correct 1 value").isEqualTo(34L); } @Test @@ -126,9 +126,9 @@ public void testReorderedProjection() throws Exception { Record projected = writeAndRead("reordered_projection", schema, reordered, record); assertThat(projected.get(0)).as("Should contain the correct 0 value").isNull(); - assertThat("test") + assertThat(projected.get(1).toString()) .as("Should contain the correct 1 value") - .isEqualTo(projected.get(1).toString()); + .isEqualTo("test"); assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); } @@ -165,9 +165,9 @@ public void testBasicProjection() throws Exception { Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); assertThat(projected.getField("data")).as("Should not project data").isNull(); - assertThat(34L) + assertThat((long) projected.getField("id")) .as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + .isEqualTo(34L); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); @@ -176,7 +176,7 @@ public void testBasicProjection() throws Exception { int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); - assertThat(0).as("Should contain the correct data value").isEqualTo(cmp); + assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); } @Test @@ -196,13 +196,13 @@ public void testRename() throws Exception { Types.NestedField.optional(1, "renamed", Types.StringType.get())); Record projected = writeAndRead("project_and_rename", writeSchema, readSchema, record); - assertThat(34L) + assertThat((long) projected.getField("id")) .as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + .isEqualTo(34L); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("renamed")); - assertThat(0).as("Should contain the correct data/renamed value").isEqualTo(cmp); + assertThat(cmp).as("Should contain the correct data/renamed value").isEqualTo(0); } @Test @@ -228,9 +228,9 @@ public void testNestedStructProjection() throws Exception { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Record projectedLocation = (Record) projected.getField("location"); - assertThat(34L) + assertThat((long) projected.getField("id")) .as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + .isEqualTo(34L); assertThat(projectedLocation).as("Should not project location").isNull(); Schema latOnly = @@ -245,9 +245,9 @@ public void testNestedStructProjection() throws Exception { assertThat(projected.getField("id")).as("Should not project id").isNull(); assertThat(projected.getField("location")).as("Should project location").isNotNull(); assertThat(projectedLocation.getField("long")).as("Should not project longitude").isNull(); - assertThat(52.995143f) + assertThat((float) projectedLocation.getField("lat")) .as("Should project latitude") - .isCloseTo((float) projectedLocation.getField("lat"), within(0.000001f)); + .isCloseTo(52.995143f, within(0.000001f)); Schema longOnly = new Schema( @@ -262,9 +262,9 @@ public void testNestedStructProjection() throws Exception { assertThat(projected.getField("id")).as("Should not project id").isNull(); assertThat(projected.getField("location")).as("Should project location").isNotNull(); assertThat(projectedLocation.getField("lat")).as("Should not project latitude").isNull(); - assertThat(-1.539054f) + assertThat((float) projectedLocation.getField("long")) .as("Should project longitude") - .isCloseTo((float) projectedLocation.getField("long"), within(0.000001f)); + .isCloseTo(-1.539054f, within(0.000001f)); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); @@ -272,12 +272,12 @@ public void testNestedStructProjection() throws Exception { assertThat(projected.getField("id")).as("Should not project id").isNull(); assertThat(projected.getField("location")).as("Should project location").isNotNull(); - assertThat(52.995143f) + assertThat((float) projectedLocation.getField("lat")) .as("Should project latitude") - .isCloseTo((float) projectedLocation.getField("lat"), within(0.000001f)); - assertThat(-1.539054f) + .isCloseTo(52.995143f, within(0.000001f)); + assertThat((float) projectedLocation.getField("long")) .as("Should project longitude") - .isCloseTo((float) projectedLocation.getField("long"), within(0.000001f)); + .isCloseTo(-1.539054f, within(0.000001f)); } @Test @@ -299,31 +299,31 @@ public void testMapProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat(34L) + assertThat((long) projected.getField("id")) .as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + .isEqualTo(34L); assertThat(projected.getField("properties")).as("Should not project properties map").isNull(); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(properties) + assertThat(toStringMap((Map) projected.getField("properties"))) .as("Should project entire map") - .isEqualTo(toStringMap((Map) projected.getField("properties"))); + .isEqualTo(properties); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(properties) + assertThat(toStringMap((Map) projected.getField("properties"))) .as("Should project entire map") - .isEqualTo(toStringMap((Map) projected.getField("properties"))); + .isEqualTo(properties); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(properties) + assertThat(toStringMap((Map) projected.getField("properties"))) .as("Should project entire map") - .isEqualTo(toStringMap((Map) projected.getField("properties"))); + .isEqualTo(properties); } private Map toStringMap(Map map) { @@ -374,31 +374,31 @@ public void testMapOfStructsProjection() throws IOException { projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(record.getField("locations")) + assertThat(toStringMap((Map) projected.getField("locations"))) .as("Should project locations map") - .isEqualTo(toStringMap((Map) projected.getField("locations"))); + .isEqualTo(record.getField("locations")); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); assertThat(projected.getField("id")).as("Should not project id").isNull(); Map locations = toStringMap((Map) projected.getField("locations")); assertThat(locations).as("Should project locations map").isNotNull(); - assertThat(Sets.newHashSet("L1", "L2")) + assertThat(locations.keySet()) .as("Should contain L1 and L2") - .isEqualTo(locations.keySet()); + .isEqualTo(Sets.newHashSet("L1", "L2")); Record projectedL1 = (Record) locations.get("L1"); assertThat(projectedL1).as("L1 should not be null").isNotNull(); - assertThat(53.992811f) + assertThat((float) projectedL1.getField("lat")) .as("L1 should contain lat") - .isCloseTo((float) projectedL1.getField("lat"), within(0.000001f)); + .isCloseTo(53.992811f, within(0.000001f)); assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); Record projectedL2 = (Record) locations.get("L2"); assertThat(projectedL2).as("L2 should not be null").isNotNull(); - assertThat(52.995143f) + assertThat((float) projectedL2.getField("lat")) .as("L2 should contain lat") - .isCloseTo((float) projectedL2.getField("lat"), within(0.000001f)); + .isCloseTo(52.995143f, within(0.000001f)); assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); projected = @@ -407,23 +407,23 @@ public void testMapOfStructsProjection() throws IOException { locations = toStringMap((Map) projected.getField("locations")); assertThat(locations).as("Should project locations map").isNotNull(); - assertThat(Sets.newHashSet("L1", "L2")) + assertThat(locations.keySet()) .as("Should contain L1 and L2") - .isEqualTo(locations.keySet()); + .isEqualTo(Sets.newHashSet("L1", "L2")); projectedL1 = (Record) locations.get("L1"); assertThat(projectedL1).as("L1 should not be null").isNotNull(); assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); - assertThat(-1.542616f) + assertThat((float) projectedL1.getField("long")) .as("L1 should contain long") - .isCloseTo((float) projectedL1.getField("long"), within(0.000001f)); + .isCloseTo(-1.542616f, within(0.000001f)); projectedL2 = (Record) locations.get("L2"); assertThat(projectedL2).as("L2 should not be null").isNotNull(); assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); - assertThat(-1.539054f) + assertThat((float) projectedL2.getField("long")) .as("L2 should contain long") - .isCloseTo((float) projectedL2.getField("long"), within(0.000001f)); + .isCloseTo(-1.539054f, within(0.000001f)); Schema latitiudeRenamed = new Schema( @@ -441,23 +441,23 @@ public void testMapOfStructsProjection() throws IOException { assertThat(projected.getField("id")).as("Should not project id").isNull(); locations = toStringMap((Map) projected.getField("locations")); assertThat(locations).as("Should project locations map").isNotNull(); - assertThat(Sets.newHashSet("L1", "L2")) + assertThat(locations.keySet()) .as("Should contain L1 and L2") - .isEqualTo(locations.keySet()); + .isEqualTo(Sets.newHashSet("L1", "L2")); projectedL1 = (Record) locations.get("L1"); assertThat(projectedL1).as("L1 should not be null").isNotNull(); - assertThat(53.992811f) + assertThat((float) projectedL1.getField("latitude")) .as("L1 should contain latitude") - .isCloseTo((float) projectedL1.getField("latitude"), within(0.000001f)); + .isCloseTo(53.992811f, within(0.000001f)); assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); projectedL2 = (Record) locations.get("L2"); assertThat(projectedL2).as("L2 should not be null").isNotNull(); - assertThat(52.995143f) + assertThat((float) projectedL2.getField("latitude")) .as("L2 should contain latitude") - .isCloseTo((float) projectedL2.getField("latitude"), within(0.000001f)); + .isCloseTo(52.995143f, within(0.000001f)); assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); } @@ -479,20 +479,20 @@ public void testListProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat(34L) + assertThat((long) projected.getField("id")) .as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + .isEqualTo(34L); assertThat(projected.getField("values")).as("Should not project values list").isNull(); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(values).as("Should project entire list").isEqualTo(projected.getField("values")); + assertThat(projected.getField("values")).as("Should project entire list").isEqualTo(values); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(values).as("Should project entire list").isEqualTo(projected.getField("values")); + assertThat(projected.getField("values")).as("Should project entire list").isEqualTo(values); } @Test @@ -523,16 +523,16 @@ public void testListOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat(34L) + assertThat((long) projected.getField("id")) .as("Should contain the correct id value") - .isEqualTo((long) projected.getField("id")); + .isEqualTo(34L); assertThat(projected.getField("points")).as("Should not project points list").isNull(); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); assertThat(projected.getField("id")).as("Should not project id").isNull(); - assertThat(record.getField("points")) + assertThat(projected.getField("points")) .as("Should project points list") - .isEqualTo(projected.getField("points")); + .isEqualTo(record.getField("points")); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); assertThat(projected.getField("id")).as("Should not project id").isNull(); @@ -542,11 +542,11 @@ public void testListOfStructsProjection() throws IOException { assertThat(points).as("Should read 2 points").hasSize(2); Record projectedP1 = points.get(0); - assertThat(1).as("Should project x").isEqualTo((int) projectedP1.getField("x")); + assertThat((int) projectedP1.getField("x")).as("Should project x").isEqualTo(1); assertThat(projected.getField("y")).as("Should not project y").isNull(); Record projectedP2 = points.get(1); - assertThat(3).as("Should project x").isEqualTo((int) projectedP2.getField("x")); + assertThat((int) projectedP2.getField("x")).as("Should project x").isEqualTo(3); assertThat(projected.getField("y")).as("Should not project y").isNull(); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); @@ -558,7 +558,7 @@ public void testListOfStructsProjection() throws IOException { projectedP1 = points.get(0); assertThat(projectedP1.getField("x")).as("Should not project x").isNull(); - assertThat(2).as("Should project y").isEqualTo((int) projectedP1.getField("y")); + assertThat((int) projectedP1.getField("y")).as("Should project y").isEqualTo(2); projectedP2 = points.get(1); assertThat(projectedP2.getField("x")).as("Should not project x").isNull(); @@ -584,7 +584,7 @@ public void testListOfStructsProjection() throws IOException { projectedP1 = points.get(0); assertThat(projectedP1.getField("x")).as("Should not project x").isNull(); assertThat(projectedP1.getField("y")).as("Should not project y").isNull(); - assertThat(2).as("Should project z").isEqualTo((int) projectedP1.getField("z")); + assertThat((int) projectedP1.getField("z")).as("Should project z").isEqualTo(2); projectedP2 = points.get(1); assertThat(projectedP2.getField("x")).as("Should not project x").isNull(); @@ -611,12 +611,12 @@ public void testListOfStructsProjection() throws IOException { assertThat(points).as("Should read 2 points").hasSize(2); projectedP1 = points.get(0); - assertThat(1).as("Should project x").isEqualTo((int) projectedP1.getField("x")); - assertThat(2).as("Should project y").isEqualTo((int) projectedP1.getField("y")); + assertThat((int) projectedP1.getField("x")).as("Should project x").isEqualTo(1); + assertThat((int) projectedP1.getField("y")).as("Should project y").isEqualTo(2); assertThat(projectedP1.getField("z")).as("Should contain null z").isNull(); projectedP2 = points.get(1); - assertThat(3).as("Should project x").isEqualTo((int) projectedP2.getField("x")); + assertThat((int) projectedP2.getField("x")).as("Should project x").isEqualTo(3); assertThat(projectedP2.getField("y")).as("Should project null y").isNull(); assertThat(projectedP2.getField("z")).as("Should contain null z").isNull(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java index 8d4dd9881ecb..06b68b77e680 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java @@ -51,23 +51,23 @@ public void testAggregates() { Max max = new Max(namedReference); Expression expectedMax = Expressions.max(unquoted); Expression actualMax = SparkAggregates.convert(max); - assertThat(expectedMax.toString()) + assertThat(String.valueOf(actualMax)) .as("Max must match") - .isEqualTo(String.valueOf(actualMax)); + .isEqualTo(expectedMax.toString()); Min min = new Min(namedReference); Expression expectedMin = Expressions.min(unquoted); Expression actualMin = SparkAggregates.convert(min); - assertThat(expectedMin.toString()) + assertThat(String.valueOf(actualMin)) .as("Min must match") - .isEqualTo(String.valueOf(actualMin)); + .isEqualTo(expectedMin.toString()); Count count = new Count(namedReference, false); Expression expectedCount = Expressions.count(unquoted); Expression actualCount = SparkAggregates.convert(count); - assertThat(expectedCount.toString()) + assertThat(String.valueOf(actualCount)) .as("Count must match") - .isEqualTo(String.valueOf(actualCount)); + .isEqualTo(expectedCount.toString()); Count countDistinct = new Count(namedReference, true); Expression convertedCountDistinct = SparkAggregates.convert(countDistinct); @@ -76,9 +76,9 @@ public void testAggregates() { CountStar countStar = new CountStar(); Expression expectedCountStar = Expressions.countStar(); Expression actualCountStar = SparkAggregates.convert(countStar); - assertThat(expectedCountStar.toString()) + assertThat(String.valueOf(actualCountStar)) .as("CountStar must match") - .isEqualTo(String.valueOf(actualCountStar)); + .isEqualTo(expectedCountStar.toString()); }); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java index 4a0f9b58d8e9..d37df7882d24 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java @@ -111,9 +111,9 @@ public void testTaskSetPlanning() throws NoSuchTableException, IOException { .option(SparkReadOptions.SCAN_TASK_SET_ID, setID) .option(SparkReadOptions.SPLIT_SIZE, tasks.get(0).file().fileSizeInBytes()) .load(tableName); - assertThat(2) + assertThat(scanDF.javaRDD().getNumPartitions()) .as("Num partitions should match") - .isEqualTo(scanDF.javaRDD().getNumPartitions()); + .isEqualTo(2); // load the staged file set and make sure we combine both files into a single split scanDF = @@ -123,9 +123,9 @@ public void testTaskSetPlanning() throws NoSuchTableException, IOException { .option(SparkReadOptions.SCAN_TASK_SET_ID, setID) .option(SparkReadOptions.SPLIT_SIZE, Long.MAX_VALUE) .load(tableName); - assertThat(1) + assertThat(scanDF.javaRDD().getNumPartitions()) .as("Num partitions should match") - .isEqualTo(scanDF.javaRDD().getNumPartitions()); + .isEqualTo(1); } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java index 7f7de2172692..d55e718ff2d3 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java @@ -36,10 +36,9 @@ public void testJsonConversion() { new StreamingOffset(System.currentTimeMillis(), 3L, false), new StreamingOffset(System.currentTimeMillis(), 4L, true) }; - assertThat(expected) + assertThat(Arrays.stream(expected).map(elem -> StreamingOffset.fromJson(elem.json())).toArray()) .as("StreamingOffsets should match") - .isEqualTo( - Arrays.stream(expected).map(elem -> StreamingOffset.fromJson(elem.json())).toArray()); + .isEqualTo(expected); } @Test @@ -52,6 +51,6 @@ public void testToJson() throws Exception { actual.put("scan_all_files", false); String expectedJson = expected.json(); String actualJson = JsonUtil.mapper().writeValueAsString(actual); - assertThat(expectedJson).isEqualTo(actualJson); + assertThat(actualJson).isEqualTo(expectedJson); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index 263e7c59bb20..3d9d6eb8ebaf 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -127,8 +127,8 @@ public void testStreamingWriteAppendMode() throws Exception { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - assertThat(expected).as("Number of rows should match").hasSameSizeAs(actual); - assertThat(expected).as("Result rows should match").isEqualTo(actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); assertThat(table.snapshots()).as("Number of snapshots should match").hasSize(2); } finally { for (StreamingQuery query : spark.streams().active()) { @@ -188,8 +188,8 @@ public void testStreamingWriteCompleteMode() throws Exception { List actual = result.orderBy("data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - assertThat(expected).as("Number of rows should match").hasSameSizeAs(actual); - assertThat(expected).as("Result rows should match").isEqualTo(actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); assertThat(table.snapshots()).as("Number of snapshots should match").hasSize(2); } finally { for (StreamingQuery query : spark.streams().active()) { @@ -249,8 +249,8 @@ public void testStreamingWriteCompleteModeWithProjection() throws Exception { List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - assertThat(expected).as("Number of rows should match").hasSameSizeAs(actual); - assertThat(expected).as("Result rows should match").isEqualTo(actual); + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + assertThat(actual).as("Result rows should match").isEqualTo(expected); assertThat(table.snapshots()).as("Number of snapshots should match").hasSize(2); } finally { for (StreamingQuery query : spark.streams().active()) { From e63dd47f517aeb9baf2cb5a0a2119b927e8a1612 Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Wed, 20 Dec 2023 14:54:42 +0530 Subject: [PATCH 06/15] set tempDir protected when used in a subclass --- .../iceberg/spark/source/TestIcebergSourceTablesBase.java | 2 +- .../org/apache/iceberg/spark/source/TestReadProjection.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 7fd1649df759..4a03b93e6d3c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -117,7 +117,7 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("id").build(); - @TempDir private Path temp; + @TempDir protected Path temp; public abstract Table createTable( TableIdentifier ident, Schema schema, PartitionSpec spec, Map properties); 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 ce0edba8d516..d5156a0d8fac 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 @@ -50,7 +50,7 @@ public abstract class TestReadProjection { protected abstract Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; - @TempDir private Path temp; + @TempDir protected Path temp; @Test public void testFullProjection() throws Exception { From 2e7d2af51028f797255c54707042ad2357acb6ed Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Wed, 20 Dec 2023 17:32:30 +0530 Subject: [PATCH 07/15] update to junit5 --- .../org/apache/iceberg/RecordWrapperTest.java | 2 +- .../spark/source/TestInternalRowWrapper.java | 17 +++++++++-------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java b/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java index 1084958f528b..22b928d23883 100644 --- a/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java +++ b/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java @@ -24,7 +24,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeWrapper; import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public abstract class RecordWrapperTest { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java index 9e75145faff9..a0bc98200281 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Iterator; import org.apache.iceberg.RecordWrapperTest; import org.apache.iceberg.Schema; @@ -29,18 +31,17 @@ import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.util.StructLikeWrapper; import org.apache.spark.sql.catalyst.InternalRow; -import org.junit.Assert; -import org.junit.Ignore; +import org.junit.jupiter.api.Disabled; public class TestInternalRowWrapper extends RecordWrapperTest { - @Ignore + @Disabled @Override public void testTimestampWithoutZone() { // Spark does not support timestamp without zone. } - @Ignore + @Disabled @Override public void testTime() { // Spark does not support time fields. @@ -61,8 +62,8 @@ protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); for (int i = 0; i < numRecords; i++) { - Assert.assertTrue("Should have more records", actual.hasNext()); - Assert.assertTrue("Should have more InternalRow", expected.hasNext()); + assertThat(actual).as("Should have more records").hasNext(); + assertThat(expected).as("Should have more InternalRow").hasNext(); StructLike recordStructLike = recordWrapper.wrap(actual.next()); StructLike rowStructLike = rowWrapper.wrap(expected.next()); @@ -73,7 +74,7 @@ protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { expectedWrapper.set(rowStructLike)); } - Assert.assertFalse("Shouldn't have more record", actual.hasNext()); - Assert.assertFalse("Shouldn't have more InternalRow", expected.hasNext()); + assertThat(actual).as("Shouldn't have more record").isExhausted(); + assertThat(expected).as("Shouldn't have more InternalRow").isExhausted(); } } From 21839d9d8c7470161f9a7669e7683dd8ad706d46 Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Wed, 20 Dec 2023 19:23:00 +0530 Subject: [PATCH 08/15] small fixes --- .../spark/source/TestDataSourceOptions.java | 5 ++- .../source/TestIcebergSourceTablesBase.java | 4 +- .../spark/source/TestIcebergSpark.java | 43 ++++++++----------- 3 files changed, 22 insertions(+), 30 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 5bcb33e0eb4a..00c9083e25fc 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -409,8 +409,9 @@ public void testExtraSnapshotMetadata() throws IOException { Table table = tables.load(tableLocation); - assertThat(table.currentSnapshot().summary().get("extra-key")).isEqualTo("someValue"); - assertThat(table.currentSnapshot().summary().get("another-key")).isEqualTo("anotherValue"); + assertThat(table.currentSnapshot().summary()) + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); } @Test diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 4a03b93e6d3c..a9ec27fe85e5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -1990,9 +1990,7 @@ public void testRemoveOrphanFilesActionSupport() throws InterruptedException { .olderThan(System.currentTimeMillis()) .execute(); - assertThat(Iterables.isEmpty(result1.orphanFileLocations())) - .as("Should not delete any metadata files") - .isTrue(); + assertThat(result1.orphanFileLocations()).as("Should not delete any metadata files").isEmpty(); DeleteOrphanFiles.Result result2 = actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java index e579712770e5..7eff93d204e4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java @@ -63,7 +63,7 @@ public void testRegisterIntegerBucketUDF() { assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) - .isEqualTo((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)); + .isEqualTo(Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)); } @Test @@ -72,7 +72,7 @@ public void testRegisterShortBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_short_16(1S)").collectAsList(); assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) - .isEqualTo((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)); + .isEqualTo(Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)); } @Test @@ -81,7 +81,7 @@ public void testRegisterByteBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_byte_16(1Y)").collectAsList(); assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) - .isEqualTo((int) Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)); + .isEqualTo(Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1)); } @Test @@ -90,7 +90,7 @@ public void testRegisterLongBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_long_16(1L)").collectAsList(); assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) - .isEqualTo((int) Transforms.bucket(16).bind(Types.LongType.get()).apply(1L)); + .isEqualTo(Transforms.bucket(16).bind(Types.LongType.get()).apply(1L)); } @Test @@ -99,7 +99,7 @@ public void testRegisterStringBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_string_16('hello')").collectAsList(); assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) - .isEqualTo((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")); + .isEqualTo(Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")); } @Test @@ -108,7 +108,7 @@ public void testRegisterCharBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_char_16('hello')").collectAsList(); assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) - .isEqualTo((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")); + .isEqualTo(Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")); } @Test @@ -117,7 +117,7 @@ public void testRegisterVarCharBucketUDF() { List results = spark.sql("SELECT iceberg_bucket_varchar_16('hello')").collectAsList(); assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) - .isEqualTo((int) Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")); + .isEqualTo(Transforms.bucket(16).bind(Types.StringType.get()).apply("hello")); } @Test @@ -128,10 +128,9 @@ public void testRegisterDateBucketUDF() { assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) .isEqualTo( - (int) - Transforms.bucket(16) - .bind(Types.DateType.get()) - .apply(DateTimeUtils.fromJavaDate(Date.valueOf("2021-06-30")))); + Transforms.bucket(16) + .bind(Types.DateType.get()) + .apply(DateTimeUtils.fromJavaDate(Date.valueOf("2021-06-30")))); } @Test @@ -145,12 +144,10 @@ public void testRegisterTimestampBucketUDF() { assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) .isEqualTo( - (int) - Transforms.bucket(16) - .bind(Types.TimestampType.withZone()) - .apply( - DateTimeUtils.fromJavaTimestamp( - Timestamp.valueOf("2021-06-30 00:00:00.000")))); + Transforms.bucket(16) + .bind(Types.TimestampType.withZone()) + .apply( + DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2021-06-30 00:00:00.000")))); } @Test @@ -160,10 +157,9 @@ public void testRegisterBinaryBucketUDF() { assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) .isEqualTo( - (int) - Transforms.bucket(16) - .bind(Types.BinaryType.get()) - .apply(ByteBuffer.wrap(new byte[] {0x00, 0x20, 0x00, 0x1F}))); + Transforms.bucket(16) + .bind(Types.BinaryType.get()) + .apply(ByteBuffer.wrap(new byte[] {0x00, 0x20, 0x00, 0x1F}))); } @Test @@ -173,10 +169,7 @@ public void testRegisterDecimalBucketUDF() { assertThat(results).hasSize(1); assertThat(results.get(0).getInt(0)) .isEqualTo( - (int) - Transforms.bucket(16) - .bind(Types.DecimalType.of(4, 2)) - .apply(new BigDecimal("11.11"))); + Transforms.bucket(16).bind(Types.DecimalType.of(4, 2)).apply(new BigDecimal("11.11"))); } @Test From ea770b2ebd8005b15a0bf788d9c8a2517dd6996f Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Wed, 20 Dec 2023 20:50:39 +0530 Subject: [PATCH 09/15] changes to clear CI --- .../iceberg/spark/source/TestReadProjection.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) 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 d5156a0d8fac..e2019a1f0697 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,7 +24,6 @@ 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.Schema; @@ -37,8 +36,10 @@ 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.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; public abstract class TestReadProjection { final String format; @@ -50,7 +51,7 @@ public abstract class TestReadProjection { protected abstract Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; - @TempDir protected Path temp; + @Rule public TemporaryFolder temp = new TemporaryFolder(); @Test public void testFullProjection() throws Exception { @@ -71,7 +72,9 @@ public void testFullProjection() throws Exception { int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); - assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); + + // TODO: update this Assert with AssertJ + Assert.assertEquals("Should contain the correct data value", 0, cmp); } @Test From cf2da6a89aba9343fa0370de514394b06db64ca4 Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Thu, 21 Dec 2023 10:21:43 +0530 Subject: [PATCH 10/15] fix import, assertJ --- .../iceberg/spark/source/TestIcebergSourceTablesBase.java | 2 +- .../org/apache/iceberg/spark/source/TestReadProjection.java | 4 +--- .../apache/iceberg/spark/source/TestSparkReadProjection.java | 2 +- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index a9ec27fe85e5..d50a401597d2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -1556,7 +1556,7 @@ public void testPartitionsTableLastUpdatedSnapshot() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2") .collectAsList(); - assertThat(filtered).as("Actual results should have one row").hasSize(2); + assertThat(filtered).as("Actual results should have one row").hasSize(1); TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); 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 e2019a1f0697..81a46cd68122 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 @@ -36,7 +36,6 @@ 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.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -73,8 +72,7 @@ public void testFullProjection() throws Exception { int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); - // TODO: update this Assert with AssertJ - Assert.assertEquals("Should contain the correct data value", 0, cmp); + assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); } @Test 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 50e6bf7364e8..3a4b235c46e5 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 @@ -110,7 +110,7 @@ public static void stopSpark() { @Override protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException { - File parent = temp.resolve(desc).toFile(); + File parent = temp.newFolder(desc); File location = new File(parent, "test"); File dataFolder = new File(location, "data"); Assert.assertTrue("mkdirs should succeed", dataFolder.mkdirs()); From 0af6802b097436968cf43440caf0292b5503552c Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Thu, 21 Dec 2023 20:07:21 +0530 Subject: [PATCH 11/15] add ParameterizedTestExtension to CatalogTestBase CatalogTestBase is supposed to replace SparkCatalogTestBase, which is parameterized. But, parameterize tests were never added to CatalogTestBase. This commit introduces it. --- .../apache/iceberg/spark/CatalogTestBase.java | 43 ++++++++++++------- 1 file changed, 27 insertions(+), 16 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java index dbb839eacc48..61f100ca5c09 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java @@ -18,29 +18,40 @@ */ package org.apache.iceberg.spark; +import java.nio.file.Path; import java.util.Map; -import java.util.stream.Stream; -import org.junit.jupiter.params.provider.Arguments; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +@ExtendWith(ParameterizedTestExtension.class) public abstract class CatalogTestBase extends TestBaseWithCatalog { // these parameters are broken out to avoid changes that need to modify lots of test suites - public static Stream parameters() { - return Stream.of( - Arguments.of( - SparkCatalogConfig.HIVE.catalogName(), - SparkCatalogConfig.HIVE.implementation(), - SparkCatalogConfig.HIVE.properties()), - Arguments.of( - SparkCatalogConfig.HADOOP.catalogName(), - SparkCatalogConfig.HADOOP.implementation(), - SparkCatalogConfig.HADOOP.properties()), - Arguments.of( - SparkCatalogConfig.SPARK.catalogName(), - SparkCatalogConfig.SPARK.implementation(), - SparkCatalogConfig.SPARK.properties())); + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties() + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties() + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties() + } + }; } + @TempDir protected Path temp; + public CatalogTestBase(SparkCatalogConfig config) { super(config); } From 8b84fdc7b6eed9197abebb76ad063d8ce737375a Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Thu, 21 Dec 2023 20:08:04 +0530 Subject: [PATCH 12/15] fix CI errors --- .../apache/iceberg/spark/source/TestSparkStagedScan.java | 6 +++--- .../org/apache/iceberg/spark/source/TestSparkTable.java | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java index d37df7882d24..3305b9e91384 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java @@ -35,7 +35,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; public class TestSparkStagedScan extends CatalogTestBase { @@ -49,7 +49,7 @@ public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testTaskSetLoading() throws NoSuchTableException, IOException { sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); @@ -84,7 +84,7 @@ public void testTaskSetLoading() throws NoSuchTableException, IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testTaskSetPlanning() throws NoSuchTableException, IOException { sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java index 21f4cc4120fd..ecb720d06e55 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java @@ -28,7 +28,7 @@ import org.apache.spark.sql.connector.catalog.TableCatalog; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; public class TestSparkTable extends CatalogTestBase { @@ -46,7 +46,7 @@ public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testTableEquality() throws NoSuchTableException { CatalogManager catalogManager = spark.sessionState().catalogManager(); TableCatalog catalog = (TableCatalog) catalogManager.catalog(catalogName); From 27500f24e4e5f212716c7ad14c8336200593c1a9 Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Fri, 22 Dec 2023 11:17:49 +0530 Subject: [PATCH 13/15] wip --- .../org/apache/iceberg/spark/source/TestChangelogReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java index 934926e17740..248bf742fbdd 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java @@ -254,6 +254,6 @@ private Object[] toJava(InternalRow row) { private DataFile writeDataFile(List records) throws IOException { // records all use IDs that are in bucket id_bucket=0 return FileHelpers.writeDataFile( - table, Files.localOutput(temp.toFile()), TestHelpers.Row.of(0), records); + table, Files.localOutput(temp.resolve("junit").toFile()), TestHelpers.Row.of(0), records); } } From a818fbb8e778be54f70479e5fe351a7f6c0b419e Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Fri, 22 Dec 2023 11:39:45 +0530 Subject: [PATCH 14/15] fix CI errors --- .../apache/iceberg/spark/source/TestChangelogReader.java | 6 +++++- .../iceberg/spark/source/TestIcebergSourceTablesBase.java | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java index 248bf742fbdd..52d6ff8c9c8b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java @@ -22,6 +22,7 @@ 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; import java.nio.file.Path; import java.util.List; @@ -254,6 +255,9 @@ private Object[] toJava(InternalRow row) { private DataFile writeDataFile(List records) throws IOException { // records all use IDs that are in bucket id_bucket=0 return FileHelpers.writeDataFile( - table, Files.localOutput(temp.resolve("junit").toFile()), TestHelpers.Row.of(0), records); + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + records); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index d50a401597d2..29ccba5a27c7 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -2281,7 +2281,7 @@ private DeleteFile writeEqDeleteFile(Table table) { try { return FileHelpers.writeDeleteFile( table, - Files.localOutput(temp.toFile()), + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), org.apache.iceberg.TestHelpers.Row.of(1), deletes, deleteRowSchema); From 90cdf81b94138b31672477402a21d0da1d65bd0d Mon Sep 17 00:00:00 2001 From: chinmay-bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Fri, 22 Dec 2023 15:51:02 +0530 Subject: [PATCH 15/15] more fluent assert calls --- .../spark/source/TestWriteMetricsConfig.java | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java index cf759a827e60..841268a6be0e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java @@ -215,10 +215,8 @@ public void testCustomMetricCollectionForParquet() throws IOException { DataFile file = task.file(); assertThat(file.nullValueCounts()).hasSize(2); assertThat(file.valueCounts()).hasSize(2); - assertThat(file.lowerBounds()).hasSize(1); - assertThat(file.lowerBounds()).containsKey(id.fieldId()); - assertThat(file.upperBounds()).hasSize(1); - assertThat(file.upperBounds()).containsKey(id.fieldId()); + assertThat(file.lowerBounds()).hasSize(1).containsKey(id.fieldId()); + assertThat(file.upperBounds()).hasSize(1).containsKey(id.fieldId()); } } @@ -271,27 +269,27 @@ public void testCustomMetricCollectionForNestedParquet() throws IOException { DataFile file = task.file(); Map nullValueCounts = file.nullValueCounts(); - assertThat(nullValueCounts).hasSize(3); - assertThat(nullValueCounts).containsKey(longCol.fieldId()); - assertThat(nullValueCounts).containsKey(recordId.fieldId()); - assertThat(nullValueCounts).containsKey(recordData.fieldId()); + assertThat(nullValueCounts) + .hasSize(3) + .containsKey(longCol.fieldId()) + .containsKey(recordId.fieldId()) + .containsKey(recordData.fieldId()); Map valueCounts = file.valueCounts(); - assertThat(valueCounts).hasSize(3); - assertThat(valueCounts).containsKey(longCol.fieldId()); - assertThat(valueCounts).containsKey(recordId.fieldId()); - assertThat(valueCounts).containsKey(recordData.fieldId()); + assertThat(valueCounts) + .hasSize(3) + .containsKey(longCol.fieldId()) + .containsKey(recordId.fieldId()) + .containsKey(recordData.fieldId()); Map lowerBounds = file.lowerBounds(); - assertThat(lowerBounds).hasSize(2); - assertThat(lowerBounds).containsKey(recordId.fieldId()); + assertThat(lowerBounds).hasSize(2).containsKey(recordId.fieldId()); ByteBuffer recordDataLowerBound = lowerBounds.get(recordData.fieldId()); assertThat(ByteBuffers.toByteArray(recordDataLowerBound)).hasSize(2); Map upperBounds = file.upperBounds(); - assertThat(upperBounds).hasSize(2); - assertThat(upperBounds).containsKey(recordId.fieldId()); + assertThat(upperBounds).hasSize(2).containsKey(recordId.fieldId()); ByteBuffer recordDataUpperBound = upperBounds.get(recordData.fieldId()); assertThat(ByteBuffers.toByteArray(recordDataUpperBound)).hasSize(2);