From 53a1c8671dd1b9b93f4a857230008c812d79ddbf Mon Sep 17 00:00:00 2001 From: Chinmay Bhat <12948588+chinmay-bhat@users.noreply.github.com> Date: Wed, 10 Jan 2024 21:35:08 +0530 Subject: [PATCH] Spark 3.5: Migrate tests in SQL directory to JUnit5 (#9401) --- .../spark/sql/PartitionedWritesTestBase.java | 103 ++-- .../spark/sql/TestAggregatePushDown.java | 156 +++--- .../iceberg/spark/sql/TestAlterTable.java | 151 +++--- .../iceberg/spark/sql/TestCreateTable.java | 262 +++++----- .../spark/sql/TestCreateTableAsSelect.java | 176 +++---- .../iceberg/spark/sql/TestDeleteFrom.java | 36 +- .../iceberg/spark/sql/TestDropTable.java | 69 +-- .../iceberg/spark/sql/TestFilterPushDown.java | 83 ++-- .../iceberg/spark/sql/TestNamespaceSQL.java | 231 +++++---- .../spark/sql/TestPartitionedWrites.java | 10 +- .../sql/TestPartitionedWritesAsSelect.java | 66 ++- .../sql/TestPartitionedWritesToBranch.java | 10 +- .../sql/TestPartitionedWritesToWapBranch.java | 20 +- .../iceberg/spark/sql/TestRefreshTable.java | 21 +- .../apache/iceberg/spark/sql/TestSelect.java | 168 ++++--- .../spark/sql/TestSparkBucketFunction.java | 244 +++++----- .../spark/sql/TestSparkDaysFunction.java | 102 ++-- .../spark/sql/TestSparkHoursFunction.java | 77 ++- .../spark/sql/TestSparkMonthsFunction.java | 106 ++--- .../spark/sql/TestSparkTruncateFunction.java | 446 +++++++++--------- .../spark/sql/TestSparkYearsFunction.java | 108 ++--- .../sql/TestStoragePartitionedJoins.java | 125 ++--- .../spark/sql/TestTimestampWithoutZone.java | 65 ++- .../spark/sql/TestUnpartitionedWrites.java | 10 +- .../sql/TestUnpartitionedWritesToBranch.java | 15 +- .../sql/UnpartitionedWritesTestBase.java | 104 ++-- 26 files changed, 1502 insertions(+), 1462 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java index 77dccbf1e064..97f8e6142dc5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java @@ -18,30 +18,26 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Arrays; import java.util.List; -import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.source.SimpleRecord; 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.functions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public abstract class PartitionedWritesTestBase extends SparkCatalogTestBase { - public PartitionedWritesTestBase( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public abstract class PartitionedWritesTestBase extends CatalogTestBase { - @Before + @BeforeEach public void createTables() { sql( "CREATE TABLE %s (id bigint, data string) USING iceberg PARTITIONED BY (truncate(id, 3))", @@ -49,22 +45,22 @@ public void createTables() { sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testInsertAppend() { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows after insert") + .isEqualTo(3L); sql("INSERT INTO %s VALUES (4, 'd'), (5, 'e')", commitTarget()); - Assert.assertEquals( - "Should have 5 rows after insert", - 5L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows after insert") + .isEqualTo(5L); List expected = ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e")); @@ -75,18 +71,18 @@ public void testInsertAppend() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testInsertOverwrite() { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows after insert") + .isEqualTo(3L); // 4 and 5 replace 3 in the partition (id - (id % 3)) = 3 sql("INSERT OVERWRITE %s VALUES (4, 'd'), (5, 'e')", commitTarget()); - Assert.assertEquals( - "Should have 4 rows after overwrite", - 4L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 4 rows after overwrite") + .isEqualTo(4L); List expected = ImmutableList.of(row(1L, "a"), row(2L, "b"), row(4L, "d"), row(5L, "e")); @@ -97,20 +93,20 @@ public void testInsertOverwrite() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDataFrameV2Append() throws NoSuchTableException { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); ds.writeTo(commitTarget()).append(); - Assert.assertEquals( - "Should have 5 rows after insert", - 5L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows after insert") + .isEqualTo(5L); List expected = ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e")); @@ -121,20 +117,20 @@ public void testDataFrameV2Append() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); ds.writeTo(commitTarget()).overwritePartitions(); - Assert.assertEquals( - "Should have 4 rows after overwrite", - 4L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 4 rows after overwrite") + .isEqualTo(4L); List expected = ImmutableList.of(row(1L, "a"), row(2L, "b"), row(4L, "d"), row(5L, "e")); @@ -145,20 +141,20 @@ public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDataFrameV2Overwrite() throws NoSuchTableException { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); ds.writeTo(commitTarget()).overwrite(functions.col("id").$less(3)); - Assert.assertEquals( - "Should have 3 rows after overwrite", - 3L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows after overwrite") + .isEqualTo(3L); List expected = ImmutableList.of(row(3L, "c"), row(4L, "d"), row(5L, "e")); @@ -168,10 +164,11 @@ public void testDataFrameV2Overwrite() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testViewsReturnRecentResults() { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); Dataset query = spark.sql("SELECT * FROM " + commitTarget() + " WHERE id = 1"); query.createOrReplaceTempView("tmp"); @@ -207,7 +204,7 @@ protected void assertPartitionMetadata( rowsToJava(actualPartitionRows.collectAsList())); } - @Test + @TestTemplate public void testWriteWithOutputSpec() throws NoSuchTableException { Table table = validationCatalog.loadTable(tableIdent); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java index 37ae96a248ef..97821dbf731e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; import java.util.List; import java.util.Locale; -import java.util.Map; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; @@ -32,35 +33,29 @@ import org.apache.iceberg.hive.TestHiveMetastore; 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.SparkCatalogTestBase; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.spark.sql.SparkSession; -import org.junit.After; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -public class TestAggregatePushDown extends SparkCatalogTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; - public TestAggregatePushDown( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +public class TestAggregatePushDown extends CatalogTestBase { - @BeforeClass + @BeforeAll public static void startMetastoreAndSpark() { - SparkTestBase.metastore = new TestHiveMetastore(); + TestBase.metastore = new TestHiveMetastore(); metastore.start(); - SparkTestBase.hiveConf = metastore.hiveConf(); + TestBase.hiveConf = metastore.hiveConf(); - SparkTestBase.spark = + TestBase.spark = SparkSession.builder() .master("local[2]") .config("spark.sql.iceberg.aggregate_pushdown", "true") .enableHiveSupport() .getOrCreate(); - SparkTestBase.catalog = + TestBase.catalog = (HiveCatalog) CatalogUtil.loadCatalog( HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); @@ -72,17 +67,17 @@ public static void startMetastoreAndSpark() { } } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testDifferentDataTypesAggregatePushDownInPartitionedTable() { testDifferentDataTypesAggregatePushDown(true); } - @Test + @TestTemplate public void testDifferentDataTypesAggregatePushDownInNonPartitionedTable() { testDifferentDataTypesAggregatePushDown(false); } @@ -148,8 +143,9 @@ private void testDifferentDataTypesAggregatePushDown(boolean hasPartitionCol) { explainContainsPushDownAggregates = true; } - Assert.assertTrue( - "explain should contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); List actual = sql(select, tableName); List expected = Lists.newArrayList(); @@ -181,7 +177,7 @@ private void testDifferentDataTypesAggregatePushDown(boolean hasPartitionCol) { assertEquals("min/max/count push down", expected, actual); } - @Test + @TestTemplate public void testDateAndTimestampWithPartition() { sql( "CREATE TABLE %s (id bigint, data string, d date, ts timestamp) USING iceberg PARTITIONED BY (id)", @@ -208,8 +204,9 @@ public void testDateAndTimestampWithPartition() { explainContainsPushDownAggregates = true; } - Assert.assertTrue( - "explain should contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); List actual = sql(select, tableName); List expected = Lists.newArrayList(); @@ -225,7 +222,7 @@ public void testDateAndTimestampWithPartition() { assertEquals("min/max/count push down", expected, actual); } - @Test + @TestTemplate public void testAggregateNotPushDownIfOneCantPushDown() { sql("CREATE TABLE %s (id LONG, data DOUBLE) USING iceberg", tableName); sql( @@ -240,8 +237,9 @@ public void testAggregateNotPushDownIfOneCantPushDown() { explainContainsPushDownAggregates = true; } - Assert.assertFalse( - "explain should not contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); List actual = sql(select, tableName); List expected = Lists.newArrayList(); @@ -249,7 +247,7 @@ public void testAggregateNotPushDownIfOneCantPushDown() { assertEquals("expected and actual should equal", expected, actual); } - @Test + @TestTemplate public void testAggregatePushDownWithMetricsMode() { sql("CREATE TABLE %s (id LONG, data DOUBLE) USING iceberg", tableName); sql( @@ -275,8 +273,9 @@ public void testAggregatePushDownWithMetricsMode() { } // count(data) is not pushed down because the metrics mode is `none` - Assert.assertFalse( - "explain should not contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); List actual1 = sql(select1, tableName); List expected1 = Lists.newArrayList(); @@ -291,8 +290,9 @@ public void testAggregatePushDownWithMetricsMode() { } // count(id) is pushed down because the metrics mode is `counts` - Assert.assertTrue( - "explain should contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); List actual2 = sql(select2, tableName); List expected2 = Lists.newArrayList(); @@ -309,8 +309,9 @@ public void testAggregatePushDownWithMetricsMode() { // COUNT(id), MAX(id) are not pushed down because MAX(id) is not pushed down (metrics mode is // `counts`) - Assert.assertFalse( - "explain should not contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); List actual3 = sql(select3, tableName); List expected3 = Lists.newArrayList(); @@ -318,7 +319,7 @@ public void testAggregatePushDownWithMetricsMode() { assertEquals("expected and actual should equal", expected3, actual3); } - @Test + @TestTemplate public void testAggregateNotPushDownForStringType() { sql("CREATE TABLE %s (id LONG, data STRING) USING iceberg", tableName); sql( @@ -337,8 +338,9 @@ public void testAggregateNotPushDownForStringType() { explainContainsPushDownAggregates = true; } - Assert.assertFalse( - "explain should not contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); List actual1 = sql(select1, tableName); List expected1 = Lists.newArrayList(); @@ -352,8 +354,9 @@ public void testAggregateNotPushDownForStringType() { explainContainsPushDownAggregates = true; } - Assert.assertTrue( - "explain should contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); List actual2 = sql(select2, tableName); List expected2 = Lists.newArrayList(); @@ -371,8 +374,9 @@ public void testAggregateNotPushDownForStringType() { explainContainsPushDownAggregates = true; } - Assert.assertTrue( - "explain should contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); List actual3 = sql(select3, tableName); List expected3 = Lists.newArrayList(); @@ -380,12 +384,12 @@ public void testAggregateNotPushDownForStringType() { assertEquals("expected and actual should equal", expected3, actual3); } - @Test + @TestTemplate public void testAggregatePushDownWithDataFilter() { testAggregatePushDownWithFilter(false); } - @Test + @TestTemplate public void testAggregatePushDownWithPartitionFilter() { testAggregatePushDownWithFilter(true); } @@ -421,13 +425,14 @@ private void testAggregatePushDownWithFilter(boolean partitionFilerOnly) { if (!partitionFilerOnly) { // Filters are not completely pushed down, we can't push down aggregates - Assert.assertFalse( - "explain should not contain the pushed down aggregates", - explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); } else { // Filters are not completely pushed down, we can push down aggregates - Assert.assertTrue( - "explain should contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); } List actual = sql(select, tableName); @@ -436,7 +441,7 @@ private void testAggregatePushDownWithFilter(boolean partitionFilerOnly) { assertEquals("expected and actual should equal", expected, actual); } - @Test + @TestTemplate public void testAggregateWithComplexType() { sql("CREATE TABLE %s (id INT, complex STRUCT) USING iceberg", tableName); sql( @@ -451,8 +456,9 @@ public void testAggregateWithComplexType() { explainContainsPushDownAggregates = true; } - Assert.assertFalse( - "count not pushed down for complex types", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("count not pushed down for complex types") + .isFalse(); List actual = sql(select1, tableName); List expected = Lists.newArrayList(); @@ -467,10 +473,12 @@ public void testAggregateWithComplexType() { explainContainsPushDownAggregates = true; } - Assert.assertFalse("max not pushed down for complex types", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("max not pushed down for complex types") + .isFalse(); } - @Test + @TestTemplate public void testAggregatePushDownInDeleteCopyOnWrite() { sql("CREATE TABLE %s (id LONG, data INT) USING iceberg", tableName); sql( @@ -488,7 +496,9 @@ public void testAggregatePushDownInDeleteCopyOnWrite() { explainContainsPushDownAggregates = true; } - Assert.assertTrue("min/max/count pushed down for deleted", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("min/max/count pushed down for deleted") + .isTrue(); List actual = sql(select, tableName); List expected = Lists.newArrayList(); @@ -496,7 +506,7 @@ public void testAggregatePushDownInDeleteCopyOnWrite() { assertEquals("min/max/count push down", expected, actual); } - @Test + @TestTemplate public void testAggregatePushDownForTimeTravel() { sql("CREATE TABLE %s (id LONG, data INT) USING iceberg", tableName); sql( @@ -516,7 +526,7 @@ public void testAggregatePushDownForTimeTravel() { if (explainString1.contains("count(id)")) { explainContainsPushDownAggregates1 = true; } - Assert.assertTrue("count pushed down", explainContainsPushDownAggregates1); + assertThat(explainContainsPushDownAggregates1).as("count pushed down").isTrue(); List actual1 = sql("SELECT count(id) FROM %s VERSION AS OF %s", tableName, snapshotId); @@ -529,13 +539,13 @@ public void testAggregatePushDownForTimeTravel() { explainContainsPushDownAggregates2 = true; } - Assert.assertTrue("count pushed down", explainContainsPushDownAggregates2); + assertThat(explainContainsPushDownAggregates2).as("count pushed down").isTrue(); List actual2 = sql("SELECT count(id) FROM %s", tableName); assertEquals("count push down", expected2, actual2); } - @Test + @TestTemplate public void testAllNull() { sql("CREATE TABLE %s (id int, data int) USING iceberg PARTITIONED BY (id)", tableName); sql( @@ -557,8 +567,9 @@ public void testAllNull() { explainContainsPushDownAggregates = true; } - Assert.assertTrue( - "explain should contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); List actual = sql(select, tableName); List expected = Lists.newArrayList(); @@ -566,7 +577,7 @@ public void testAllNull() { assertEquals("min/max/count push down", expected, actual); } - @Test + @TestTemplate public void testAllNaN() { sql("CREATE TABLE %s (id int, data float) USING iceberg PARTITIONED BY (id)", tableName); sql( @@ -588,8 +599,9 @@ public void testAllNaN() { explainContainsPushDownAggregates = true; } - Assert.assertFalse( - "explain should not contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); List actual = sql(select, tableName); List expected = Lists.newArrayList(); @@ -597,7 +609,7 @@ public void testAllNaN() { assertEquals("expected and actual should equal", expected, actual); } - @Test + @TestTemplate public void testNaN() { sql("CREATE TABLE %s (id int, data float) USING iceberg PARTITIONED BY (id)", tableName); sql( @@ -619,8 +631,9 @@ public void testNaN() { explainContainsPushDownAggregates = true; } - Assert.assertFalse( - "explain should not contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); List actual = sql(select, tableName); List expected = Lists.newArrayList(); @@ -628,7 +641,7 @@ public void testNaN() { assertEquals("expected and actual should equal", expected, actual); } - @Test + @TestTemplate public void testInfinity() { sql( "CREATE TABLE %s (id int, data1 float, data2 double, data3 double) USING iceberg PARTITIONED BY (id)", @@ -659,8 +672,9 @@ public void testInfinity() { explainContainsPushDownAggregates = true; } - Assert.assertTrue( - "explain should contain the pushed down aggregates", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); List actual = sql(select, tableName); List expected = Lists.newArrayList(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java index 89857a65e23e..7c98888f1667 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java @@ -20,42 +20,36 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; -import java.util.Map; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; import org.apache.spark.SparkException; import org.apache.spark.sql.AnalysisException; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -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.TestTemplate; -public class TestAlterTable extends SparkCatalogTestBase { +public class TestAlterTable extends CatalogTestBase { private final TableIdentifier renamedIdent = TableIdentifier.of(Namespace.of("default"), "table2"); - public TestAlterTable(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); sql("DROP TABLE IF EXISTS %s2", tableName); } - @Test + @TestTemplate public void testAddColumnNotNull() { assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN c3 INT NOT NULL", tableName)) .isInstanceOf(SparkException.class) @@ -63,7 +57,7 @@ public void testAddColumnNotNull() { "Unsupported table change: Incompatible change: cannot add required column: c3"); } - @Test + @TestTemplate public void testAddColumn() { sql( "ALTER TABLE %s ADD COLUMN point struct AFTER id", @@ -80,10 +74,9 @@ public void testAddColumn() { NestedField.required(5, "y", Types.DoubleType.get()))), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); sql("ALTER TABLE %s ADD COLUMN point.z double COMMENT 'May be null' FIRST", tableName); @@ -99,13 +92,12 @@ public void testAddColumn() { NestedField.required(5, "y", Types.DoubleType.get()))), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema2, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema2); } - @Test + @TestTemplate public void testAddColumnWithArray() { sql("ALTER TABLE %s ADD COLUMN data2 array>", tableName); // use the implicit column name 'element' to access member of array and add column d to struct. @@ -124,13 +116,12 @@ public void testAddColumnWithArray() { NestedField.optional(6, "b", Types.IntegerType.get()), NestedField.optional(7, "c", Types.IntegerType.get()), NestedField.optional(8, "d", Types.IntegerType.get()))))); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); } - @Test + @TestTemplate public void testAddColumnWithMap() { sql("ALTER TABLE %s ADD COLUMN data2 map, struct>", tableName); // use the implicit column name 'key' and 'value' to access member of map. @@ -151,10 +142,9 @@ public void testAddColumnWithMap() { NestedField.optional(7, "a", Types.IntegerType.get()), NestedField.optional(8, "b", Types.IntegerType.get()), NestedField.optional(9, "c", Types.IntegerType.get()))))); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); // should not allow changing map key column assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN data2.key.y int", tableName)) @@ -162,20 +152,19 @@ public void testAddColumnWithMap() { .hasMessageStartingWith("Unsupported table change: Cannot add fields to map keys:"); } - @Test + @TestTemplate public void testDropColumn() { sql("ALTER TABLE %s DROP COLUMN data", tableName); Types.StructType expectedSchema = Types.StructType.of(NestedField.required(1, "id", Types.LongType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); } - @Test + @TestTemplate public void testRenameColumn() { sql("ALTER TABLE %s RENAME COLUMN id TO row_id", tableName); @@ -184,13 +173,12 @@ public void testRenameColumn() { NestedField.required(1, "row_id", Types.LongType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); } - @Test + @TestTemplate public void testAlterColumnComment() { sql("ALTER TABLE %s ALTER COLUMN id COMMENT 'Record id'", tableName); @@ -199,13 +187,12 @@ public void testAlterColumnComment() { NestedField.required(1, "id", Types.LongType.get(), "Record id"), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); } - @Test + @TestTemplate public void testAlterColumnType() { sql("ALTER TABLE %s ADD COLUMN count int", tableName); sql("ALTER TABLE %s ALTER COLUMN count TYPE bigint", tableName); @@ -216,13 +203,12 @@ public void testAlterColumnType() { NestedField.optional(2, "data", Types.StringType.get()), NestedField.optional(3, "count", Types.LongType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); } - @Test + @TestTemplate public void testAlterColumnDropNotNull() { sql("ALTER TABLE %s ALTER COLUMN id DROP NOT NULL", tableName); @@ -231,13 +217,12 @@ public void testAlterColumnDropNotNull() { NestedField.optional(1, "id", Types.LongType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); } - @Test + @TestTemplate public void testAlterColumnSetNotNull() { // no-op changes are allowed sql("ALTER TABLE %s ALTER COLUMN id SET NOT NULL", tableName); @@ -247,17 +232,16 @@ public void testAlterColumnSetNotNull() { NestedField.required(1, "id", Types.LongType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); assertThatThrownBy(() -> sql("ALTER TABLE %s ALTER COLUMN data SET NOT NULL", tableName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Cannot change nullable column to non-nullable: data"); } - @Test + @TestTemplate public void testAlterColumnPositionAfter() { sql("ALTER TABLE %s ADD COLUMN count int", tableName); sql("ALTER TABLE %s ALTER COLUMN count AFTER id", tableName); @@ -268,13 +252,12 @@ public void testAlterColumnPositionAfter() { NestedField.optional(3, "count", Types.IntegerType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); } - @Test + @TestTemplate public void testAlterColumnPositionFirst() { sql("ALTER TABLE %s ADD COLUMN count int", tableName); sql("ALTER TABLE %s ALTER COLUMN count FIRST", tableName); @@ -285,27 +268,31 @@ public void testAlterColumnPositionFirst() { NestedField.required(1, "id", Types.LongType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Schema should match expected", - expectedSchema, - validationCatalog.loadTable(tableIdent).schema().asStruct()); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); } - @Test + @TestTemplate public void testTableRename() { - Assume.assumeFalse( - "Hadoop catalog does not support rename", validationCatalog instanceof HadoopCatalog); + assumeThat(validationCatalog) + .as("Hadoop catalog does not support rename") + .isNotInstanceOf(HadoopCatalog.class); - Assert.assertTrue("Initial name should exist", validationCatalog.tableExists(tableIdent)); - Assert.assertFalse("New name should not exist", validationCatalog.tableExists(renamedIdent)); + assertThat(validationCatalog.tableExists(tableIdent)).as("Initial name should exist").isTrue(); + assertThat(validationCatalog.tableExists(renamedIdent)) + .as("New name should not exist") + .isFalse(); sql("ALTER TABLE %s RENAME TO %s2", tableName, tableName); - Assert.assertFalse("Initial name should not exist", validationCatalog.tableExists(tableIdent)); - Assert.assertTrue("New name should exist", validationCatalog.tableExists(renamedIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Initial name should not exist") + .isFalse(); + assertThat(validationCatalog.tableExists(renamedIdent)).as("New name should exist").isTrue(); } - @Test + @TestTemplate public void testSetTableProperties() { sql("ALTER TABLE %s SET TBLPROPERTIES ('prop'='value')", tableName); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index ecfd6759b900..718233c697a1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.io.File; -import java.util.Map; +import java.nio.file.Files; import org.apache.iceberg.BaseTable; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -27,87 +30,92 @@ import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; - -public class TestCreateTable extends SparkCatalogTestBase { - public TestCreateTable(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestCreateTable extends CatalogTestBase { - @After + @AfterEach public void dropTestTable() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testTransformIgnoreCase() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE IF NOT EXISTS %s (id BIGINT NOT NULL, ts timestamp) " + "USING iceberg partitioned by (HOURS(ts))", tableName); - Assert.assertTrue("Table should already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should already exist").isTrue(); sql( "CREATE TABLE IF NOT EXISTS %s (id BIGINT NOT NULL, ts timestamp) " + "USING iceberg partitioned by (hours(ts))", tableName); - Assert.assertTrue("Table should already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should already exist").isTrue(); } - @Test + @TestTemplate public void testTransformSingularForm() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE IF NOT EXISTS %s (id BIGINT NOT NULL, ts timestamp) " + "USING iceberg partitioned by (hour(ts))", tableName); - Assert.assertTrue("Table should exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should exist").isTrue(); } - @Test + @TestTemplate public void testTransformPluralForm() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE IF NOT EXISTS %s (id BIGINT NOT NULL, ts timestamp) " + "USING iceberg partitioned by (hours(ts))", tableName); - Assert.assertTrue("Table should exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should exist").isTrue(); } - @Test + @TestTemplate public void testCreateTable() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql("CREATE TABLE %s (id BIGINT NOT NULL, data STRING) USING iceberg", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertNotNull("Should load the new table", table); + assertThat(table).as("Should load the new table").isNotNull(); StructType expectedSchema = StructType.of( NestedField.required(1, "id", Types.LongType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Should have the expected schema", expectedSchema, table.schema().asStruct()); - Assert.assertEquals("Should not be partitioned", 0, table.spec().fields().size()); - Assert.assertNull( - "Should not have the default format set", - table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").hasSize(0); + assertThat(table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)) + .as("Should not have the default format set") + .isNull(); } - @Test + @TestTemplate public void testCreateTableInRootNamespace() { - Assume.assumeTrue( - "Hadoop has no default namespace configured", "testhadoop".equals(catalogName)); + assumeThat(catalogName) + .as("Hadoop has no default namespace configured") + .isEqualTo("testhadoop"); try { sql("CREATE TABLE %s.table (id bigint) USING iceberg", catalogName); @@ -116,30 +124,32 @@ public void testCreateTableInRootNamespace() { } } - @Test + @TestTemplate public void testCreateTableUsingParquet() { - Assume.assumeTrue( - "Not working with session catalog because Spark will not use v2 for a Parquet table", - !"spark_catalog".equals(catalogName)); + assumeThat(catalogName) + .as("Not working with session catalog because Spark will not use v2 for a Parquet table") + .isNotEqualTo("spark_catalog"); - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql("CREATE TABLE %s (id BIGINT NOT NULL, data STRING) USING parquet", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertNotNull("Should load the new table", table); + assertThat(table).as("Should load the new table").isNotNull(); StructType expectedSchema = StructType.of( NestedField.required(1, "id", Types.LongType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Should have the expected schema", expectedSchema, table.schema().asStruct()); - Assert.assertEquals("Should not be partitioned", 0, table.spec().fields().size()); - Assert.assertEquals( - "Should not have default format parquet", - "parquet", - table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").hasSize(0); + assertThat(table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)) + .as("Should not have default format parquet") + .isEqualTo("parquet"); Assertions.assertThatThrownBy( () -> @@ -150,9 +160,11 @@ public void testCreateTableUsingParquet() { .hasMessage("Unsupported format in USING: crocodile"); } - @Test + @TestTemplate public void testCreateTablePartitionedBy() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE %s " @@ -162,7 +174,7 @@ public void testCreateTablePartitionedBy() { tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertNotNull("Should load the new table", table); + assertThat(table).as("Should load the new table").isNotNull(); StructType expectedSchema = StructType.of( @@ -170,8 +182,9 @@ public void testCreateTablePartitionedBy() { NestedField.optional(2, "created_at", Types.TimestampType.withZone()), NestedField.optional(3, "category", Types.StringType.get()), NestedField.optional(4, "data", Types.StringType.get())); - Assert.assertEquals( - "Should have the expected schema", expectedSchema, table.schema().asStruct()); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); PartitionSpec expectedSpec = PartitionSpec.builderFor(new Schema(expectedSchema.fields())) @@ -179,16 +192,18 @@ public void testCreateTablePartitionedBy() { .bucket("id", 8) .day("created_at") .build(); - Assert.assertEquals("Should be partitioned correctly", expectedSpec, table.spec()); + assertThat(table.spec()).as("Should be partitioned correctly").isEqualTo(expectedSpec); - Assert.assertNull( - "Should not have the default format set", - table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); + assertThat(table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)) + .as("Should not have the default format set") + .isNull(); } - @Test + @TestTemplate public void testCreateTableColumnComments() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE %s " @@ -197,23 +212,26 @@ public void testCreateTableColumnComments() { tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertNotNull("Should load the new table", table); + assertThat(table).as("Should load the new table").isNotNull(); StructType expectedSchema = StructType.of( NestedField.required(1, "id", Types.LongType.get(), "Unique identifier"), NestedField.optional(2, "data", Types.StringType.get(), "Data value")); - Assert.assertEquals( - "Should have the expected schema", expectedSchema, table.schema().asStruct()); - Assert.assertEquals("Should not be partitioned", 0, table.spec().fields().size()); - Assert.assertNull( - "Should not have the default format set", - table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").hasSize(0); + assertThat(table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)) + .as("Should not have the default format set") + .isNull(); } - @Test + @TestTemplate public void testCreateTableComment() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE %s " @@ -223,34 +241,36 @@ public void testCreateTableComment() { tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertNotNull("Should load the new table", table); + assertThat(table).as("Should load the new table").isNotNull(); StructType expectedSchema = StructType.of( NestedField.required(1, "id", Types.LongType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Should have the expected schema", expectedSchema, table.schema().asStruct()); - Assert.assertEquals("Should not be partitioned", 0, table.spec().fields().size()); - Assert.assertNull( - "Should not have the default format set", - table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); - Assert.assertEquals( - "Should have the table comment set in properties", - "Table doc", - table.properties().get(TableCatalog.PROP_COMMENT)); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").hasSize(0); + assertThat(table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)) + .as("Should not have the default format set") + .isNull(); + assertThat(table.properties().get(TableCatalog.PROP_COMMENT)) + .as("Should have the table comment set in properties") + .isEqualTo("Table doc"); } - @Test + @TestTemplate public void testCreateTableLocation() throws Exception { - Assume.assumeTrue( - "Cannot set custom locations for Hadoop catalog tables", - !(validationCatalog instanceof HadoopCatalog)); + assumeThat(validationCatalog) + .as("Cannot set custom locations for Hadoop catalog tables") + .isNotInstanceOf(HadoopCatalog.class); - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); - File tableLocation = temp.newFolder(); - Assert.assertTrue(tableLocation.delete()); + File tableLocation = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableLocation.delete()).isTrue(); String location = "file:" + tableLocation.toString(); @@ -262,24 +282,27 @@ public void testCreateTableLocation() throws Exception { tableName, location); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertNotNull("Should load the new table", table); + assertThat(table).as("Should load the new table").isNotNull(); StructType expectedSchema = StructType.of( NestedField.required(1, "id", Types.LongType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Should have the expected schema", expectedSchema, table.schema().asStruct()); - Assert.assertEquals("Should not be partitioned", 0, table.spec().fields().size()); - Assert.assertNull( - "Should not have the default format set", - table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); - Assert.assertEquals("Should have a custom table location", location, table.location()); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").hasSize(0); + assertThat(table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)) + .as("Should not have the default format set") + .isNull(); + assertThat(table.location()).as("Should have a custom table location").isEqualTo(location); } - @Test + @TestTemplate public void testCreateTableProperties() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE %s " @@ -289,22 +312,24 @@ public void testCreateTableProperties() { tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertNotNull("Should load the new table", table); + assertThat(table).as("Should load the new table").isNotNull(); StructType expectedSchema = StructType.of( NestedField.required(1, "id", Types.LongType.get()), NestedField.optional(2, "data", Types.StringType.get())); - Assert.assertEquals( - "Should have the expected schema", expectedSchema, table.schema().asStruct()); - Assert.assertEquals("Should not be partitioned", 0, table.spec().fields().size()); - Assert.assertEquals("Should have property p1", "2", table.properties().get("p1")); - Assert.assertEquals("Should have property p2", "x", table.properties().get("p2")); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").hasSize(0); + assertThat(table.properties()).containsEntry("p1", "2").containsEntry("p2", "x"); } - @Test + @TestTemplate public void testCreateTableWithFormatV2ThroughTableProperty() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE %s " @@ -314,15 +339,16 @@ public void testCreateTableWithFormatV2ThroughTableProperty() { tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "should create table using format v2", - 2, - ((BaseTable) table).operations().current().formatVersion()); + assertThat(((BaseTable) table).operations().current().formatVersion()) + .as("should create table using format v2") + .isEqualTo(2); } - @Test + @TestTemplate public void testUpgradeTableWithFormatV2ThroughTableProperty() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE %s " @@ -333,15 +359,21 @@ public void testUpgradeTableWithFormatV2ThroughTableProperty() { Table table = validationCatalog.loadTable(tableIdent); TableOperations ops = ((BaseTable) table).operations(); - Assert.assertEquals("should create table using format v1", 1, ops.refresh().formatVersion()); + assertThat(ops.refresh().formatVersion()) + .as("should create table using format v1") + .isEqualTo(1); sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='2')", tableName); - Assert.assertEquals("should update table to use format v2", 2, ops.refresh().formatVersion()); + assertThat(ops.refresh().formatVersion()) + .as("should update table to use format v2") + .isEqualTo(2); } - @Test + @TestTemplate public void testDowngradeTableToFormatV1ThroughTablePropertyFails() { - Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); sql( "CREATE TABLE %s " @@ -352,7 +384,9 @@ public void testDowngradeTableToFormatV1ThroughTablePropertyFails() { Table table = validationCatalog.loadTable(tableIdent); TableOperations ops = ((BaseTable) table).operations(); - Assert.assertEquals("should create table using format v2", 2, ops.refresh().formatVersion()); + assertThat(ops.refresh().formatVersion()) + .as("should create table using format v2") + .isEqualTo(2); Assertions.assertThatThrownBy( () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='1')", tableName)) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java index 1c08b1fd5ae7..81b193942394 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java @@ -21,27 +21,51 @@ import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.functions.lit; import static org.apache.spark.sql.functions.when; +import static org.assertj.core.api.Assertions.assertThat; -import java.util.Map; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; - -public class TestCreateTableAsSelect extends SparkCatalogTestBase { - - private final String sourceName; - - public TestCreateTableAsSelect( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - this.sourceName = tableName("source"); +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestCreateTableAsSelect extends CatalogTestBase { + + @Parameter(index = 3) + private String sourceName; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, sourceName = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + SparkCatalogConfig.HIVE.catalogName() + ".default.source" + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + SparkCatalogConfig.HADOOP.catalogName() + ".default.source" + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + "default.source" + } + }; + } + @BeforeEach + public void createTableIfNotExists() { sql( "CREATE TABLE IF NOT EXISTS %s (id bigint NOT NULL, data string) " + "USING iceberg PARTITIONED BY (truncate(id, 3))", @@ -49,12 +73,12 @@ public TestCreateTableAsSelect( sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", sourceName); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testUnpartitionedCTAS() { sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", tableName, sourceName); @@ -65,18 +89,18 @@ public void testUnpartitionedCTAS() { Table ctasTable = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Should have expected nullable schema", - expectedSchema.asStruct(), - ctasTable.schema().asStruct()); - Assert.assertEquals("Should be an unpartitioned table", 0, ctasTable.spec().fields().size()); + assertThat(ctasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + + assertThat(ctasTable.spec().fields()).as("Should be an unpartitioned table").hasSize(0); assertEquals( "Should have rows matching the source table", sql("SELECT * FROM %s ORDER BY id", sourceName), sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testPartitionedCTAS() { sql( "CREATE TABLE %s USING iceberg PARTITIONED BY (id) AS SELECT * FROM %s ORDER BY id", @@ -91,18 +115,17 @@ public void testPartitionedCTAS() { Table ctasTable = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Should have expected nullable schema", - expectedSchema.asStruct(), - ctasTable.schema().asStruct()); - Assert.assertEquals("Should be partitioned by id", expectedSpec, ctasTable.spec()); + assertThat(ctasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(ctasTable.spec()).as("Should be partitioned by id").isEqualTo(expectedSpec); assertEquals( "Should have rows matching the source table", sql("SELECT * FROM %s ORDER BY id", sourceName), sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRTAS() { sql( "CREATE TABLE %s USING iceberg TBLPROPERTIES ('prop1'='val1', 'prop2'='val2')" @@ -132,11 +155,10 @@ public void testRTAS() { Table rtasTable = validationCatalog.loadTable(tableIdent); // the replacement table has a different schema and partition spec than the original - Assert.assertEquals( - "Should have expected nullable schema", - expectedSchema.asStruct(), - rtasTable.schema().asStruct()); - Assert.assertEquals("Should be partitioned by part", expectedSpec, rtasTable.spec()); + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(rtasTable.spec()).as("Should be partitioned by part").isEqualTo(expectedSpec); assertEquals( "Should have rows matching the source table", @@ -146,18 +168,19 @@ public void testRTAS() { sourceName), sql("SELECT * FROM %s ORDER BY id", tableName)); - Assert.assertEquals( - "Table should have expected snapshots", 2, Iterables.size(rtasTable.snapshots())); - - Assert.assertEquals( - "Should have updated table property", "newval1", rtasTable.properties().get("prop1")); - Assert.assertEquals( - "Should have preserved table property", "val2", rtasTable.properties().get("prop2")); - Assert.assertEquals( - "Should have new table property", "val3", rtasTable.properties().get("prop3")); + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); + assertThat(rtasTable.properties().get("prop1")) + .as("Should have updated table property") + .isEqualTo("newval1"); + assertThat(rtasTable.properties().get("prop2")) + .as("Should have preserved table property") + .isEqualTo("val2"); + assertThat(rtasTable.properties().get("prop3")) + .as("Should have new table property") + .isEqualTo("val3"); } - @Test + @TestTemplate public void testCreateRTAS() { sql( "CREATE OR REPLACE TABLE %s USING iceberg PARTITIONED BY (part) AS " @@ -194,11 +217,10 @@ public void testCreateRTAS() { Table rtasTable = validationCatalog.loadTable(tableIdent); // the replacement table has a different schema and partition spec than the original - Assert.assertEquals( - "Should have expected nullable schema", - expectedSchema.asStruct(), - rtasTable.schema().asStruct()); - Assert.assertEquals("Should be partitioned by part", expectedSpec, rtasTable.spec()); + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(rtasTable.spec()).as("Should be partitioned by part").isEqualTo(expectedSpec); assertEquals( "Should have rows matching the source table", @@ -208,11 +230,10 @@ public void testCreateRTAS() { sourceName), sql("SELECT * FROM %s ORDER BY id", tableName)); - Assert.assertEquals( - "Table should have expected snapshots", 2, Iterables.size(rtasTable.snapshots())); + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); } - @Test + @TestTemplate public void testDataFrameV2Create() throws Exception { spark.table(sourceName).writeTo(tableName).using("iceberg").create(); @@ -223,18 +244,17 @@ public void testDataFrameV2Create() throws Exception { Table ctasTable = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Should have expected nullable schema", - expectedSchema.asStruct(), - ctasTable.schema().asStruct()); - Assert.assertEquals("Should be an unpartitioned table", 0, ctasTable.spec().fields().size()); + assertThat(ctasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(ctasTable.spec().fields()).as("Should be an unpartitioned table").hasSize(0); assertEquals( "Should have rows matching the source table", sql("SELECT * FROM %s ORDER BY id", sourceName), sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testDataFrameV2Replace() throws Exception { spark.table(sourceName).writeTo(tableName).using("iceberg").create(); @@ -267,11 +287,10 @@ public void testDataFrameV2Replace() throws Exception { Table rtasTable = validationCatalog.loadTable(tableIdent); // the replacement table has a different schema and partition spec than the original - Assert.assertEquals( - "Should have expected nullable schema", - expectedSchema.asStruct(), - rtasTable.schema().asStruct()); - Assert.assertEquals("Should be partitioned by part", expectedSpec, rtasTable.spec()); + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(rtasTable.spec()).as("Should be partitioned by part").isEqualTo(expectedSpec); assertEquals( "Should have rows matching the source table", @@ -281,11 +300,10 @@ public void testDataFrameV2Replace() throws Exception { sourceName), sql("SELECT * FROM %s ORDER BY id", tableName)); - Assert.assertEquals( - "Table should have expected snapshots", 2, Iterables.size(rtasTable.snapshots())); + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); } - @Test + @TestTemplate public void testDataFrameV2CreateOrReplace() { spark .table(sourceName) @@ -335,11 +353,10 @@ public void testDataFrameV2CreateOrReplace() { Table rtasTable = validationCatalog.loadTable(tableIdent); // the replacement table has a different schema and partition spec than the original - Assert.assertEquals( - "Should have expected nullable schema", - expectedSchema.asStruct(), - rtasTable.schema().asStruct()); - Assert.assertEquals("Should be partitioned by part", expectedSpec, rtasTable.spec()); + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(rtasTable.spec()).as("Should be partitioned by part").isEqualTo(expectedSpec); assertEquals( "Should have rows matching the source table", @@ -349,11 +366,10 @@ public void testDataFrameV2CreateOrReplace() { sourceName), sql("SELECT * FROM %s ORDER BY id", tableName)); - Assert.assertEquals( - "Table should have expected snapshots", 2, Iterables.size(rtasTable.snapshots())); + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); } - @Test + @TestTemplate public void testCreateRTASWithPartitionSpecChanging() { sql( "CREATE OR REPLACE TABLE %s USING iceberg PARTITIONED BY (part) AS " @@ -393,13 +409,12 @@ public void testCreateRTASWithPartitionSpecChanging() { .withSpecId(2) // The Spec is new .build(); - Assert.assertEquals("Should be partitioned by part and id", expectedSpec, rtasTable.spec()); + assertThat(rtasTable.spec()).as("Should be partitioned by part and id").isEqualTo(expectedSpec); // the replacement table has a different schema and partition spec than the original - Assert.assertEquals( - "Should have expected nullable schema", - expectedSchema.asStruct(), - rtasTable.schema().asStruct()); + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); assertEquals( "Should have rows matching the source table", @@ -409,7 +424,6 @@ public void testCreateRTASWithPartitionSpecChanging() { sourceName), sql("SELECT * FROM %s ORDER BY id", tableName)); - Assert.assertEquals( - "Table should have expected snapshots", 2, Iterables.size(rtasTable.snapshots())); + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java index a4b1ef3b9df6..a68f6e239b16 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java @@ -18,33 +18,28 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; -import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; - -public class TestDeleteFrom extends SparkCatalogTestBase { - public TestDeleteFrom(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; - @After +public class TestDeleteFrom extends CatalogTestBase { + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testDeleteFromUnpartitionedTable() throws NoSuchTableException { sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); @@ -74,7 +69,7 @@ public void testDeleteFromUnpartitionedTable() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testDeleteFromTableAtSnapshot() throws NoSuchTableException { sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); @@ -92,7 +87,7 @@ public void testDeleteFromTableAtSnapshot() throws NoSuchTableException { .hasMessageStartingWith("Cannot delete from table at a specific snapshot"); } - @Test + @TestTemplate public void testDeleteFromPartitionedTable() throws NoSuchTableException { sql( "CREATE TABLE %s (id bigint, data string) " @@ -125,7 +120,7 @@ public void testDeleteFromPartitionedTable() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testDeleteFromWhereFalse() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); @@ -136,17 +131,16 @@ public void testDeleteFromWhereFalse() { sql("SELECT * FROM %s ORDER BY id", tableName)); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 1 snapshot", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 1 snapshot").hasSize(1); sql("DELETE FROM %s WHERE false", tableName); table.refresh(); - Assert.assertEquals( - "Delete should not produce a new snapshot", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Delete should not produce a new snapshot").hasSize(1); } - @Test + @TestTemplate public void testTruncate() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); @@ -157,7 +151,7 @@ public void testTruncate() { sql("SELECT * FROM %s ORDER BY id", tableName)); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 1 snapshot", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 1 snapshot").hasSize(1); sql("TRUNCATE TABLE %s", tableName); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java index 496e7a618ba0..01d8686db768 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -28,37 +29,32 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - -public class TestDropTable extends SparkCatalogTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; - public TestDropTable(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +public class TestDropTable extends CatalogTestBase { - @Before + @BeforeEach public void createTable() { sql("CREATE TABLE %s (id INT, name STRING) USING iceberg", tableName); sql("INSERT INTO %s VALUES (1, 'test')", tableName); } - @After + @AfterEach public void removeTable() throws IOException { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testDropTable() throws IOException { dropTableInternal(); } - @Test + @TestTemplate public void testDropTableGCDisabled() throws IOException { sql("ALTER TABLE %s SET TBLPROPERTIES (gc.enabled = false)", tableName); dropTableInternal(); @@ -71,23 +67,26 @@ private void dropTableInternal() throws IOException { sql("SELECT * FROM %s", tableName)); List manifestAndFiles = manifestsAndFiles(); - Assert.assertEquals( - "There should be 2 files for manifests and files", 2, manifestAndFiles.size()); - Assert.assertTrue("All files should be existed", checkFilesExist(manifestAndFiles, true)); + assertThat(manifestAndFiles).as("There should be 2 files for manifests and files").hasSize(2); + assertThat(checkFilesExist(manifestAndFiles, true)).as("All files should exist").isTrue(); sql("DROP TABLE %s", tableName); - Assert.assertFalse("Table should not exist", validationCatalog.tableExists(tableIdent)); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should not exist").isFalse(); if (catalogName.equals("testhadoop")) { // HadoopCatalog drop table without purge will delete the base table location. - Assert.assertTrue("All files should be deleted", checkFilesExist(manifestAndFiles, false)); + assertThat(checkFilesExist(manifestAndFiles, false)) + .as("All files should be deleted") + .isTrue(); } else { - Assert.assertTrue("All files should not be deleted", checkFilesExist(manifestAndFiles, true)); + assertThat(checkFilesExist(manifestAndFiles, true)) + .as("All files should not be deleted") + .isTrue(); } } // TODO: enable once SPARK-43203 is fixed - @Ignore + @Disabled public void testPurgeTable() throws IOException { assertEquals( "Should have expected rows", @@ -95,17 +94,16 @@ public void testPurgeTable() throws IOException { sql("SELECT * FROM %s", tableName)); List manifestAndFiles = manifestsAndFiles(); - Assert.assertEquals( - "There should be 2 files for manifests and files", 2, manifestAndFiles.size()); - Assert.assertTrue("All files should exist", checkFilesExist(manifestAndFiles, true)); + assertThat(manifestAndFiles).as("There should be 2 files for manifests and files").hasSize(2); + assertThat(checkFilesExist(manifestAndFiles, true)).as("All files should exist").isTrue(); sql("DROP TABLE %s PURGE", tableName); - Assert.assertFalse("Table should not exist", validationCatalog.tableExists(tableIdent)); - Assert.assertTrue("All files should be deleted", checkFilesExist(manifestAndFiles, false)); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should not exist").isFalse(); + assertThat(checkFilesExist(manifestAndFiles, false)).as("All files should be deleted").isTrue(); } // TODO: enable once SPARK-43203 is fixed - @Ignore + @Disabled public void testPurgeTableGCDisabled() throws IOException { sql("ALTER TABLE %s SET TBLPROPERTIES (gc.enabled = false)", tableName); @@ -115,17 +113,20 @@ public void testPurgeTableGCDisabled() throws IOException { sql("SELECT * FROM %s", tableName)); List manifestAndFiles = manifestsAndFiles(); - Assert.assertEquals( - "There totally should have 2 files for manifests and files", 2, manifestAndFiles.size()); - Assert.assertTrue("All files should be existed", checkFilesExist(manifestAndFiles, true)); + assertThat(manifestAndFiles).as("There should be 2 files for manifests and files").hasSize(2); + assertThat(checkFilesExist(manifestAndFiles, true)).as("All files should exist").isTrue(); Assertions.assertThatThrownBy(() -> sql("DROP TABLE %s PURGE", tableName)) .isInstanceOf(ValidationException.class) .hasMessageContaining( "Cannot purge table: GC is disabled (deleting files may corrupt other tables"); - Assert.assertTrue("Table should not been dropped", validationCatalog.tableExists(tableIdent)); - Assert.assertTrue("All files should not be deleted", checkFilesExist(manifestAndFiles, true)); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not been dropped") + .isTrue(); + assertThat(checkFilesExist(manifestAndFiles, true)) + .as("All files should not be deleted") + .isTrue(); } private List manifestsAndFiles() { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java index f2ef2d4705ea..9d2ce2b388a2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java @@ -20,44 +20,57 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; import java.math.BigDecimal; import java.sql.Timestamp; import java.time.Instant; import java.util.List; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.spark.sql.execution.SparkPlan; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFilterPushDown extends SparkTestBaseWithCatalog { - - @Parameterized.Parameters(name = "planningMode = {0}") - public static Object[] parameters() { - return new Object[] {LOCAL, DISTRIBUTED}; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFilterPushDown extends TestBaseWithCatalog { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, planningMode = {0}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + LOCAL + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + DISTRIBUTED + } + }; } - private final PlanningMode planningMode; + @Parameter(index = 3) + private PlanningMode planningMode; - public TestFilterPushDown(PlanningMode planningMode) { - this.planningMode = planningMode; - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS tmp_view"); } - @Test + @TestTemplate public void testFilterPushdownWithDecimalValues() { sql( "CREATE TABLE %s (id INT, salary DECIMAL(10, 2), dep STRING)" @@ -76,7 +89,7 @@ public void testFilterPushdownWithDecimalValues() { ImmutableList.of(row(2, new BigDecimal("100.05"), "d1"))); } - @Test + @TestTemplate public void testFilterPushdownWithIdentityTransform() { sql( "CREATE TABLE %s (id INT, salary INT, dep STRING)" @@ -188,7 +201,7 @@ public void testFilterPushdownWithIdentityTransform() { ImmutableList.of(row(5, 500, "d5"))); } - @Test + @TestTemplate public void testFilterPushdownWithHoursTransform() { sql( "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" @@ -234,7 +247,7 @@ public void testFilterPushdownWithHoursTransform() { }); } - @Test + @TestTemplate public void testFilterPushdownWithDaysTransform() { sql( "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" @@ -277,7 +290,7 @@ public void testFilterPushdownWithDaysTransform() { }); } - @Test + @TestTemplate public void testFilterPushdownWithMonthsTransform() { sql( "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" @@ -320,7 +333,7 @@ public void testFilterPushdownWithMonthsTransform() { }); } - @Test + @TestTemplate public void testFilterPushdownWithYearsTransform() { sql( "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" @@ -363,7 +376,7 @@ public void testFilterPushdownWithYearsTransform() { }); } - @Test + @TestTemplate public void testFilterPushdownWithBucketTransform() { sql( "CREATE TABLE %s (id INT, salary INT, dep STRING)" @@ -382,7 +395,7 @@ public void testFilterPushdownWithBucketTransform() { ImmutableList.of(row(1, 100, "d1"))); } - @Test + @TestTemplate public void testFilterPushdownWithTruncateTransform() { sql( "CREATE TABLE %s (id INT, salary INT, dep STRING)" @@ -407,7 +420,7 @@ public void testFilterPushdownWithTruncateTransform() { ImmutableList.of(row(1, 100, "d1"))); } - @Test + @TestTemplate public void testFilterPushdownWithSpecEvolutionAndIdentityTransforms() { sql( "CREATE TABLE %s (id INT, salary INT, dep STRING, sub_dep STRING)" @@ -448,7 +461,7 @@ public void testFilterPushdownWithSpecEvolutionAndIdentityTransforms() { ImmutableList.of(row(1, 100, "d1", "sd1"))); } - @Test + @TestTemplate public void testFilterPushdownWithSpecEvolutionAndTruncateTransform() { sql( "CREATE TABLE %s (id INT, salary INT, dep STRING)" @@ -489,7 +502,7 @@ public void testFilterPushdownWithSpecEvolutionAndTruncateTransform() { ImmutableList.of(row(1, 100, "d1"))); } - @Test + @TestTemplate public void testFilterPushdownWithSpecEvolutionAndTimeTransforms() { sql( "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" @@ -526,7 +539,7 @@ public void testFilterPushdownWithSpecEvolutionAndTimeTransforms() { }); } - @Test + @TestTemplate public void testFilterPushdownWithSpecialFloatingPointPartitionValues() { sql( "CREATE TABLE %s (id INT, salary DOUBLE)" + "USING iceberg " + "PARTITIONED BY (salary)", @@ -585,16 +598,14 @@ private void checkFilters( String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); if (sparkFilter != null) { - Assertions.assertThat(planAsString) + assertThat(planAsString) .as("Post scan filter should match") .contains("Filter (" + sparkFilter + ")"); } else { - Assertions.assertThat(planAsString) - .as("Should be no post scan filter") - .doesNotContain("Filter ("); + assertThat(planAsString).as("Should be no post scan filter").doesNotContain("Filter ("); } - Assertions.assertThat(planAsString) + assertThat(planAsString) .as("Pushed filters must match") .contains("[filters=" + icebergFilters + ","); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java index 5ee2526a714d..8661715f3e33 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java @@ -18,90 +18,131 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.io.File; import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestNamespaceSQL extends SparkCatalogTestBase { +public class TestNamespaceSQL extends CatalogTestBase { private static final Namespace NS = Namespace.of("db"); - private final String fullNamespace; - private final boolean isHadoopCatalog; - - public TestNamespaceSQL(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - this.fullNamespace = ("spark_catalog".equals(catalogName) ? "" : catalogName + ".") + NS; - this.isHadoopCatalog = "testhadoop".equals(catalogName); + @Parameter(index = 3) + private String fullNamespace; + + @Parameter(index = 4) + private boolean isHadoopCatalog; + + @Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}, fullNameSpace = {3}, isHadoopCatalog = {4}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + SparkCatalogConfig.HIVE.catalogName() + "." + NS.toString(), + false + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + SparkCatalogConfig.HADOOP.catalogName() + "." + NS.toString(), + true + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + NS.toString(), + false + } + }; } - @After + @AfterEach public void cleanNamespaces() { sql("DROP TABLE IF EXISTS %s.table", fullNamespace); sql("DROP NAMESPACE IF EXISTS %s", fullNamespace); } - @Test + @TestTemplate public void testCreateNamespace() { - Assert.assertFalse( - "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); sql("CREATE NAMESPACE %s", fullNamespace); - Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); } - @Test + @TestTemplate public void testDefaultNamespace() { - Assume.assumeFalse("Hadoop has no default namespace configured", isHadoopCatalog); + assumeThat(isHadoopCatalog).as("Hadoop has no default namespace configured").isFalse(); sql("USE %s", catalogName); Object[] current = Iterables.getOnlyElement(sql("SHOW CURRENT NAMESPACE")); - Assert.assertEquals("Should use the current catalog", current[0], catalogName); - Assert.assertEquals("Should use the configured default namespace", current[1], "default"); + assertThat(current[0]).as("Should use the current catalog").isEqualTo(catalogName); + assertThat(current[1]).as("Should use the configured default namespace").isEqualTo("default"); } - @Test + @TestTemplate public void testDropEmptyNamespace() { - Assert.assertFalse( - "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); sql("CREATE NAMESPACE %s", fullNamespace); - Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); sql("DROP NAMESPACE %s", fullNamespace); - Assert.assertFalse( - "Namespace should have been dropped", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should have been dropped") + .isFalse(); } - @Test + @TestTemplate public void testDropNonEmptyNamespace() { - Assume.assumeFalse("Session catalog has flaky behavior", "spark_catalog".equals(catalogName)); + assumeThat(catalogName).as("Session catalog has flaky behavior").isNotEqualTo("spark_catalog"); - Assert.assertFalse( - "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); sql("CREATE NAMESPACE %s", fullNamespace); sql("CREATE TABLE %s.table (id bigint) USING iceberg", fullNamespace); - Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(NS)); - Assert.assertTrue( - "Table should exist", validationCatalog.tableExists(TableIdentifier.of(NS, "table"))); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + assertThat(validationCatalog.tableExists(TableIdentifier.of(NS, "table"))) + .as("Table should exist") + .isTrue(); Assertions.assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) .isInstanceOf(NamespaceNotEmptyException.class) @@ -110,134 +151,148 @@ public void testDropNonEmptyNamespace() { sql("DROP TABLE %s.table", fullNamespace); } - @Test + @TestTemplate public void testListTables() { - Assert.assertFalse( - "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); sql("CREATE NAMESPACE %s", fullNamespace); - Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); List rows = sql("SHOW TABLES IN %s", fullNamespace); - Assert.assertEquals("Should not list any tables", 0, rows.size()); + assertThat(rows).as("Should not list any tables").hasSize(0); sql("CREATE TABLE %s.table (id bigint) USING iceberg", fullNamespace); Object[] row = Iterables.getOnlyElement(sql("SHOW TABLES IN %s", fullNamespace)); - Assert.assertEquals("Namespace should match", "db", row[0]); - Assert.assertEquals("Table name should match", "table", row[1]); + assertThat(row[0]).as("Namespace should match").isEqualTo("db"); + assertThat(row[1]).as("Table name should match").isEqualTo("table"); } - @Test + @TestTemplate public void testListNamespace() { - Assert.assertFalse( - "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); sql("CREATE NAMESPACE %s", fullNamespace); - Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); List namespaces = sql("SHOW NAMESPACES IN %s", catalogName); if (isHadoopCatalog) { - Assert.assertEquals("Should have 1 namespace", 1, namespaces.size()); + assertThat(namespaces).as("Should have 1 namespace").hasSize(1); Set namespaceNames = namespaces.stream().map(arr -> arr[0].toString()).collect(Collectors.toSet()); - Assert.assertEquals("Should have only db namespace", ImmutableSet.of("db"), namespaceNames); + assertThat(namespaceNames) + .as("Should have only db namespace") + .isEqualTo(ImmutableSet.of("db")); } else { - Assert.assertEquals("Should have 2 namespaces", 2, namespaces.size()); + assertThat(namespaces).as("Should have 2 namespaces").hasSize(2); Set namespaceNames = namespaces.stream().map(arr -> arr[0].toString()).collect(Collectors.toSet()); - Assert.assertEquals( - "Should have default and db namespaces", - ImmutableSet.of("default", "db"), - namespaceNames); + assertThat(namespaceNames) + .as("Should have default and db namespaces") + .isEqualTo(ImmutableSet.of("default", "db")); } List nestedNamespaces = sql("SHOW NAMESPACES IN %s", fullNamespace); Set nestedNames = nestedNamespaces.stream().map(arr -> arr[0].toString()).collect(Collectors.toSet()); - Assert.assertEquals("Should not have nested namespaces", ImmutableSet.of(), nestedNames); + assertThat(nestedNames).as("Should not have nested namespaces").isEmpty(); } - @Test + @TestTemplate public void testCreateNamespaceWithMetadata() { - Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - Assert.assertFalse( - "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); sql("CREATE NAMESPACE %s WITH PROPERTIES ('prop'='value')", fullNamespace); - Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); Map nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Namespace should have expected prop value", "value", nsMetadata.get("prop")); + assertThat(nsMetadata).containsEntry("prop", "value"); } - @Test + @TestTemplate public void testCreateNamespaceWithComment() { - Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - Assert.assertFalse( - "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); sql("CREATE NAMESPACE %s COMMENT 'namespace doc'", fullNamespace); - Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); Map nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Namespace should have expected comment", "namespace doc", nsMetadata.get("comment")); + assertThat(nsMetadata).containsEntry("comment", "namespace doc"); } - @Test + @TestTemplate public void testCreateNamespaceWithLocation() throws Exception { - Assume.assumeFalse("HadoopCatalog does not support namespace locations", isHadoopCatalog); + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - Assert.assertFalse( - "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); - File location = temp.newFile(); - Assert.assertTrue(location.delete()); + File location = File.createTempFile("junit", null, temp.toFile()); + assertThat(location.delete()).isTrue(); sql("CREATE NAMESPACE %s LOCATION '%s'", fullNamespace, location); - Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); Map nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Namespace should have expected location", - "file:" + location.getPath(), - nsMetadata.get("location")); + assertThat(nsMetadata).containsEntry("location", "file:" + location.getPath()); } - @Test + @TestTemplate public void testSetProperties() { - Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - Assert.assertFalse( - "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); sql("CREATE NAMESPACE %s", fullNamespace); - Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(NS)); + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); Map defaultMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); - Assert.assertFalse( - "Default metadata should not have custom property", defaultMetadata.containsKey("prop")); + assertThat(defaultMetadata) + .as("Default metadata should not have custom property") + .doesNotContainKey("prop"); sql("ALTER NAMESPACE %s SET PROPERTIES ('prop'='value')", fullNamespace); Map nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Namespace should have expected prop value", "value", nsMetadata.get("prop")); + assertThat(nsMetadata).containsEntry("prop", "value"); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java index a18bd997250b..800d17dd4559 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java @@ -18,12 +18,4 @@ */ package org.apache.iceberg.spark.sql; -import java.util.Map; - -public class TestPartitionedWrites extends PartitionedWritesTestBase { - - public TestPartitionedWrites( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } -} +public class TestPartitionedWrites extends PartitionedWritesTestBase {} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java index 3ffd38b83c3b..373ca9996efd 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java @@ -18,34 +18,51 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.IntStream; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; import org.apache.iceberg.spark.IcebergSpark; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.spark.sql.types.DataTypes; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class TestPartitionedWritesAsSelect extends SparkTestBaseWithCatalog { - - private final String targetTable = tableName("target_table"); +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestPartitionedWritesAsSelect extends TestBaseWithCatalog { + + @Parameter(index = 3) + private String targetTable; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, targetTable = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + SparkCatalogConfig.HADOOP.catalogName() + ".default.target_table" + }, + }; + } - @Before + @BeforeEach public void createTables() { sql( "CREATE TABLE %s (id bigint, data string, category string, ts timestamp) USING iceberg", tableName); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s", targetTable); } - @Test + @TestTemplate public void testInsertAsSelectAppend() { insertData(3); List expected = currentData(); @@ -58,10 +75,9 @@ public void testInsertAsSelectAppend() { sql( "INSERT INTO %s SELECT id, data, category, ts FROM %s ORDER BY ts,category", targetTable, tableName); - Assert.assertEquals( - "Should have 15 rows after insert", - 3 * 5L, - scalarSql("SELECT count(*) FROM %s", targetTable)); + assertThat(scalarSql("SELECT count(*) FROM %s", targetTable)) + .as("Should have 15 rows after insert") + .isEqualTo(3 * 5L); assertEquals( "Row data should match expected", @@ -69,7 +85,7 @@ public void testInsertAsSelectAppend() { sql("SELECT * FROM %s ORDER BY id", targetTable)); } - @Test + @TestTemplate public void testInsertAsSelectWithBucket() { insertData(3); List expected = currentData(); @@ -83,10 +99,9 @@ public void testInsertAsSelectWithBucket() { sql( "INSERT INTO %s SELECT id, data, category, ts FROM %s ORDER BY iceberg_bucket8(data)", targetTable, tableName); - Assert.assertEquals( - "Should have 15 rows after insert", - 3 * 5L, - scalarSql("SELECT count(*) FROM %s", targetTable)); + assertThat(scalarSql("SELECT count(*) FROM %s", targetTable)) + .as("Should have 15 rows after insert") + .isEqualTo(3 * 5L); assertEquals( "Row data should match expected", @@ -94,7 +109,7 @@ public void testInsertAsSelectWithBucket() { sql("SELECT * FROM %s ORDER BY id", targetTable)); } - @Test + @TestTemplate public void testInsertAsSelectWithTruncate() { insertData(3); List expected = currentData(); @@ -110,10 +125,9 @@ public void testInsertAsSelectWithTruncate() { "INSERT INTO %s SELECT id, data, category, ts FROM %s " + "ORDER BY iceberg_truncate_string4(data),iceberg_truncate_long4(id)", targetTable, tableName); - Assert.assertEquals( - "Should have 15 rows after insert", - 3 * 5L, - scalarSql("SELECT count(*) FROM %s", targetTable)); + assertThat(scalarSql("SELECT count(*) FROM %s", targetTable)) + .as("Should have 15 rows after insert") + .isEqualTo(3 * 5L); assertEquals( "Row data should match expected", diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java index c6cde7a5524e..154c6181a594 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java @@ -18,20 +18,14 @@ */ package org.apache.iceberg.spark.sql; -import java.util.Map; import org.apache.iceberg.Table; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; public class TestPartitionedWritesToBranch extends PartitionedWritesTestBase { private static final String BRANCH = "test"; - public TestPartitionedWritesToBranch( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Before + @BeforeEach @Override public void createTables() { super.createTables(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java index 5dde5f33d965..0504bbc24d44 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java @@ -19,27 +19,21 @@ package org.apache.iceberg.spark.sql; import java.util.List; -import java.util.Map; import java.util.UUID; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.spark.SparkSQLProperties; import org.assertj.core.api.Assertions; -import org.junit.After; -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.TestTemplate; public class TestPartitionedWritesToWapBranch extends PartitionedWritesTestBase { private static final String BRANCH = "test"; - public TestPartitionedWritesToWapBranch( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Before + @BeforeEach @Override public void createTables() { spark.conf().set(SparkSQLProperties.WAP_BRANCH, BRANCH); @@ -49,7 +43,7 @@ public void createTables() { sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); } - @After + @AfterEach @Override public void removeTables() { super.removeTables(); @@ -67,7 +61,7 @@ protected String selectTarget() { return String.format("%s VERSION AS OF '%s'", tableName, BRANCH); } - @Test + @TestTemplate public void testBranchAndWapBranchCannotBothBeSetForWrite() { Table table = validationCatalog.loadTable(tableIdent); table.manageSnapshots().createBranch("test2", table.refs().get(BRANCH).snapshotId()).commit(); @@ -80,7 +74,7 @@ public void testBranchAndWapBranchCannotBothBeSetForWrite() { BRANCH); } - @Test + @TestTemplate public void testWapIdAndWapBranchCannotBothBeSetForWrite() { String wapId = UUID.randomUUID().toString(); spark.conf().set(SparkSQLProperties.WAP_ID, wapId); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java index 7da2dc0882db..8a9ae0f6030a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java @@ -19,34 +19,29 @@ package org.apache.iceberg.spark.sql; import java.util.List; -import java.util.Map; import org.apache.iceberg.DataFile; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalogConfig; -import org.apache.iceberg.spark.SparkCatalogTestBase; -import org.junit.After; -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.TestTemplate; -public class TestRefreshTable extends SparkCatalogTestBase { +public class TestRefreshTable extends CatalogTestBase { - public TestRefreshTable(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Before + @BeforeEach public void createTables() { sql("CREATE TABLE %s (key int, value int) USING iceberg", tableName); sql("INSERT INTO %s VALUES (1,1)", tableName); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testRefreshCommand() { // We are not allowed to change the session catalog after it has been initialized, so build a // new one diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 93bb21b41a2b..80d7d8787e27 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -18,37 +18,65 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.text.SimpleDateFormat; import java.util.Date; import java.util.List; -import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; - -public class TestSelect extends SparkCatalogTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestSelect extends CatalogTestBase { private int scanEventCount = 0; private ScanEvent lastScanEvent = null; - private String binaryTableName = tableName("binary_table"); - public TestSelect(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); + @Parameter(index = 3) + private String binaryTableName; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, binaryTableName = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + SparkCatalogConfig.HIVE.catalogName() + ".default.binary_table" + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + SparkCatalogConfig.HADOOP.catalogName() + ".default.binary_table" + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + "default.binary_table" + } + }; + } + @BeforeEach + public void createTables() { // register a scan event listener to validate pushdown Listeners.register( event -> { @@ -56,10 +84,7 @@ public TestSelect(String catalogName, String implementation, Map lastScanEvent = event; }, ScanEvent.class); - } - @Before - public void createTables() { sql("CREATE TABLE %s (id bigint, data string, float float) USING iceberg", tableName); sql("INSERT INTO %s VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', float('NaN'))", tableName); @@ -67,13 +92,13 @@ public void createTables() { this.lastScanEvent = null; } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s", binaryTableName); } - @Test + @TestTemplate public void testSelect() { List expected = ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); @@ -81,7 +106,7 @@ public void testSelect() { assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testSelectRewrite() { List expected = ImmutableList.of(row(3L, "c", Float.NaN)); @@ -90,29 +115,28 @@ public void testSelectRewrite() { expected, sql("SELECT * FROM %s where float = float('NaN')", tableName)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should push down expected filter", - "(float IS NOT NULL AND is_nan(float))", - Spark3Util.describe(lastScanEvent.filter())); + assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); + assertThat(Spark3Util.describe(lastScanEvent.filter())) + .as("Should push down expected filter") + .isEqualTo("(float IS NOT NULL AND is_nan(float))"); } - @Test + @TestTemplate public void testProjection() { List expected = ImmutableList.of(row(1L), row(2L), row(3L)); assertEquals("Should return all expected rows", expected, sql("SELECT id FROM %s", tableName)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - Assert.assertEquals( - "Should project only the id column", - validationCatalog.loadTable(tableIdent).schema().select("id").asStruct(), - lastScanEvent.projection().asStruct()); + assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); + assertThat(lastScanEvent.projection().asStruct()) + .as("Should project only the id column") + .isEqualTo(validationCatalog.loadTable(tableIdent).schema().select("id").asStruct()); } - @Test + @TestTemplate public void testExpressionPushdown() { List expected = ImmutableList.of(row("b")); @@ -121,22 +145,21 @@ public void testExpressionPushdown() { expected, sql("SELECT data FROM %s WHERE id = 2", tableName)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should push down expected filter", - "(id IS NOT NULL AND id = 2)", - Spark3Util.describe(lastScanEvent.filter())); - Assert.assertEquals( - "Should project only id and data columns", - validationCatalog.loadTable(tableIdent).schema().select("id", "data").asStruct(), - lastScanEvent.projection().asStruct()); + assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); + assertThat(Spark3Util.describe(lastScanEvent.filter())) + .as("Should push down expected filter") + .isEqualTo("(id IS NOT NULL AND id = 2)"); + assertThat(lastScanEvent.projection().asStruct()) + .as("Should project only id and data columns") + .isEqualTo( + validationCatalog.loadTable(tableIdent).schema().select("id", "data").asStruct()); } - @Test + @TestTemplate public void testMetadataTables() { - Assume.assumeFalse( - "Spark session catalog does not support metadata tables", - "spark_catalog".equals(catalogName)); + assumeThat(catalogName) + .as("Spark session catalog does not support metadata tables") + .isNotEqualTo("spark_catalog"); assertEquals( "Snapshot metadata table", @@ -144,11 +167,11 @@ public void testMetadataTables() { sql("SELECT * FROM %s.snapshots", tableName)); } - @Test + @TestTemplate public void testSnapshotInTableName() { - Assume.assumeFalse( - "Spark session catalog does not support extended table names", - "spark_catalog".equals(catalogName)); + assumeThat(catalogName) + .as("Spark session catalog does not support extended table names") + .isNotEqualTo("spark_catalog"); // get the snapshot ID of the last write and get the current row set as expected long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); @@ -173,11 +196,11 @@ public void testSnapshotInTableName() { assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); } - @Test + @TestTemplate public void testTimestampInTableName() { - Assume.assumeFalse( - "Spark session catalog does not support extended table names", - "spark_catalog".equals(catalogName)); + assumeThat(catalogName) + .as("Spark session catalog does not support extended table names") + .isNotEqualTo("spark_catalog"); // get a timestamp just after the last write and get the current row set as expected long snapshotTs = validationCatalog.loadTable(tableIdent).currentSnapshot().timestampMillis(); @@ -203,7 +226,7 @@ public void testTimestampInTableName() { assertEquals("Snapshot at timestamp " + timestamp, expected, fromDF); } - @Test + @TestTemplate public void testVersionAsOf() { // get the snapshot ID of the last write and get the current row set as expected long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); @@ -233,7 +256,7 @@ public void testVersionAsOf() { assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); } - @Test + @TestTemplate public void testTagReference() { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); @@ -264,7 +287,7 @@ public void testTagReference() { assertEquals("Snapshot at specific tag reference name", expected, fromDF); } - @Test + @TestTemplate public void testUseSnapshotIdForTagReferenceAsOf() { Table table = validationCatalog.loadTable(tableIdent); long snapshotId1 = table.currentSnapshot().snapshotId(); @@ -289,7 +312,7 @@ public void testUseSnapshotIdForTagReferenceAsOf() { assertEquals("Snapshot at specific tag reference name", actual, travelWithLongResult); } - @Test + @TestTemplate public void testBranchReference() { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); @@ -325,15 +348,14 @@ public void testBranchReference() { assertEquals("Snapshot at specific branch reference name", expected, fromDF); } - @Test + @TestTemplate public void testUnknownReferenceAsOf() { - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s VERSION AS OF 'test_unknown'", tableName)) + assertThatThrownBy(() -> sql("SELECT * FROM %s VERSION AS OF 'test_unknown'", tableName)) .hasMessageContaining("Cannot find matching snapshot ID or reference name for version") .isInstanceOf(ValidationException.class); } - @Test + @TestTemplate public void testTimestampAsOf() { long snapshotTs = validationCatalog.loadTable(tableIdent).currentSnapshot().timestampMillis(); long timestamp = waitUntilAfter(snapshotTs + 1000); @@ -380,7 +402,7 @@ public void testTimestampAsOf() { assertEquals("Snapshot at timestamp " + timestamp, expected, fromDF); } - @Test + @TestTemplate public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { // get the snapshot ID of the last write long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); @@ -395,7 +417,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // using snapshot in table identifier and VERSION AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s VERSION AS OF %s", @@ -405,7 +427,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); // using snapshot in table identifier and TIMESTAMP AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s VERSION AS OF %s", @@ -415,7 +437,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); // using timestamp in table identifier and VERSION AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s TIMESTAMP AS OF %s", @@ -425,7 +447,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); // using timestamp in table identifier and TIMESTAMP AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s TIMESTAMP AS OF %s", @@ -435,7 +457,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } - @Test + @TestTemplate public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); @@ -444,13 +466,13 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // using branch_b1 in the table identifier and VERSION AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } - @Test + @TestTemplate public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); @@ -461,7 +483,7 @@ public void testSpecifySnapshotAndTimestamp() { // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { spark .read() @@ -478,7 +500,7 @@ public void testSpecifySnapshotAndTimestamp() { snapshotId, timestamp)); } - @Test + @TestTemplate public void testBinaryInFilter() { sql("CREATE TABLE %s (id bigint, binary binary) USING iceberg", binaryTableName); sql("INSERT INTO %s VALUES (1, X''), (2, X'1111'), (3, X'11')", binaryTableName); @@ -490,7 +512,7 @@ public void testBinaryInFilter() { sql("SELECT id, binary FROM %s where binary > X'11'", binaryTableName)); } - @Test + @TestTemplate public void testComplexTypeFilter() { String complexTypeTableName = tableName("complex_table"); sql( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java index c4113408aff9..907b91177125 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java @@ -18,184 +18,174 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.spark.functions.BucketFunction; import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.types.DataTypes; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; -public class TestSparkBucketFunction extends SparkTestBaseWithCatalog { - @Before +public class TestSparkBucketFunction extends TestBaseWithCatalog { + @BeforeEach public void useCatalog() { sql("USE %s", catalogName); } - @Test + @TestTemplate public void testSpecValues() { - Assert.assertEquals( - "Spec example: hash(34) = 2017239379", - 2017239379, - new BucketFunction.BucketInt(DataTypes.IntegerType).hash(34)); - - Assert.assertEquals( - "Spec example: hash(34L) = 2017239379", - 2017239379, - new BucketFunction.BucketLong(DataTypes.LongType).hash(34L)); - - Assert.assertEquals( - "Spec example: hash(decimal2(14.20)) = -500754589", - -500754589, - new BucketFunction.BucketDecimal(DataTypes.createDecimalType(9, 2)) - .hash(new BigDecimal("14.20"))); + assertThat(new BucketFunction.BucketInt(DataTypes.IntegerType).hash(34)) + .as("Spec example: hash(34) = 2017239379") + .isEqualTo(2017239379); + + assertThat(new BucketFunction.BucketLong(DataTypes.IntegerType).hash(34L)) + .as("Spec example: hash(34L) = 2017239379") + .isEqualTo(2017239379); + + assertThat( + new BucketFunction.BucketDecimal(DataTypes.createDecimalType(9, 2)) + .hash(new BigDecimal("14.20"))) + .as("Spec example: hash(decimal2(14.20)) = -500754589") + .isEqualTo(-500754589); Literal date = Literal.of("2017-11-16").to(Types.DateType.get()); - Assert.assertEquals( - "Spec example: hash(2017-11-16) = -653330422", - -653330422, - new BucketFunction.BucketInt(DataTypes.DateType).hash(date.value())); + assertThat(new BucketFunction.BucketInt(DataTypes.DateType).hash(date.value())) + .as("Spec example: hash(2017-11-16) = -653330422") + .isEqualTo(-653330422); Literal timestampVal = Literal.of("2017-11-16T22:31:08").to(Types.TimestampType.withoutZone()); - Assert.assertEquals( - "Spec example: hash(2017-11-16T22:31:08) = -2047944441", - -2047944441, - new BucketFunction.BucketLong(DataTypes.TimestampType).hash(timestampVal.value())); + assertThat(new BucketFunction.BucketLong(DataTypes.TimestampType).hash(timestampVal.value())) + .as("Spec example: hash(2017-11-16T22:31:08) = -2047944441") + .isEqualTo(-2047944441); Literal timestampntzVal = Literal.of("2017-11-16T22:31:08").to(Types.TimestampType.withoutZone()); - Assert.assertEquals( - "Spec example: hash(2017-11-16T22:31:08) = -2047944441", - -2047944441, - new BucketFunction.BucketLong(DataTypes.TimestampNTZType).hash(timestampntzVal.value())); + assertThat( + new BucketFunction.BucketLong(DataTypes.TimestampNTZType).hash(timestampntzVal.value())) + .as("Spec example: hash(2017-11-16T22:31:08) = -2047944441") + .isEqualTo(-2047944441); - Assert.assertEquals( - "Spec example: hash(\"iceberg\") = 1210000089", - 1210000089, - new BucketFunction.BucketString().hash("iceberg")); + assertThat(new BucketFunction.BucketString().hash("iceberg")) + .as("Spec example: hash(\"iceberg\") = 1210000089") + .isEqualTo(1210000089); ByteBuffer bytes = ByteBuffer.wrap(new byte[] {0, 1, 2, 3}); - Assert.assertEquals( - "Spec example: hash([00 01 02 03]) = -188683207", - -188683207, - new BucketFunction.BucketBinary().hash(bytes)); + assertThat(new BucketFunction.BucketBinary().hash(bytes)) + .as("Spec example: hash([00 01 02 03]) = -188683207") + .isEqualTo(-188683207); } - @Test + @TestTemplate public void testBucketIntegers() { - Assert.assertEquals( - "Byte type should bucket similarly to integer", - 3, - scalarSql("SELECT system.bucket(10, 8Y)")); - Assert.assertEquals( - "Short type should bucket similarly to integer", - 3, - scalarSql("SELECT system.bucket(10, 8S)")); + assertThat(scalarSql("SELECT system.bucket(10, 8Y)")) + .as("Byte type should bucket similarly to integer") + .isEqualTo(3); + assertThat(scalarSql("SELECT system.bucket(10, 8S)")) + .as("Short type should bucket similarly to integer") + .isEqualTo(3); // Integers - Assert.assertEquals(3, scalarSql("SELECT system.bucket(10, 8)")); - Assert.assertEquals(79, scalarSql("SELECT system.bucket(100, 34)")); - Assert.assertNull(scalarSql("SELECT system.bucket(1, CAST(null AS INT))")); + assertThat(scalarSql("SELECT system.bucket(10, 8)")).isEqualTo(3); + assertThat(scalarSql("SELECT system.bucket(100, 34)")).isEqualTo(79); + assertThat(scalarSql("SELECT system.bucket(1, CAST(null AS INT))")).isNull(); } - @Test + @TestTemplate public void testBucketDates() { - Assert.assertEquals(3, scalarSql("SELECT system.bucket(10, date('1970-01-09'))")); - Assert.assertEquals(79, scalarSql("SELECT system.bucket(100, date('1970-02-04'))")); - Assert.assertNull(scalarSql("SELECT system.bucket(1, CAST(null AS DATE))")); + assertThat(scalarSql("SELECT system.bucket(10, date('1970-01-09'))")).isEqualTo(3); + assertThat(scalarSql("SELECT system.bucket(100, date('1970-02-04'))")).isEqualTo(79); + assertThat(scalarSql("SELECT system.bucket(1, CAST(null AS DATE))")).isNull(); } - @Test + @TestTemplate public void testBucketLong() { - Assert.assertEquals(79, scalarSql("SELECT system.bucket(100, 34L)")); - Assert.assertEquals(76, scalarSql("SELECT system.bucket(100, 0L)")); - Assert.assertEquals(97, scalarSql("SELECT system.bucket(100, -34L)")); - Assert.assertEquals(0, scalarSql("SELECT system.bucket(2, -1L)")); - Assert.assertNull(scalarSql("SELECT system.bucket(2, CAST(null AS LONG))")); + assertThat(scalarSql("SELECT system.bucket(100, 34L)")).isEqualTo(79); + assertThat(scalarSql("SELECT system.bucket(100, 0L)")).isEqualTo(76); + assertThat(scalarSql("SELECT system.bucket(100, -34L)")).isEqualTo(97); + assertThat(scalarSql("SELECT system.bucket(2, -1L)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.bucket(2, CAST(null AS LONG))")).isNull(); } - @Test + @TestTemplate public void testBucketDecimal() { - Assert.assertEquals(56, scalarSql("SELECT system.bucket(64, CAST('12.34' as DECIMAL(9, 2)))")); - Assert.assertEquals(13, scalarSql("SELECT system.bucket(18, CAST('12.30' as DECIMAL(9, 2)))")); - Assert.assertEquals(2, scalarSql("SELECT system.bucket(16, CAST('12.999' as DECIMAL(9, 3)))")); - Assert.assertEquals(21, scalarSql("SELECT system.bucket(32, CAST('0.05' as DECIMAL(5, 2)))")); - Assert.assertEquals(85, scalarSql("SELECT system.bucket(128, CAST('0.05' as DECIMAL(9, 2)))")); - Assert.assertEquals(3, scalarSql("SELECT system.bucket(18, CAST('0.05' as DECIMAL(9, 2)))")); - - Assert.assertNull( - "Null input should return null", - scalarSql("SELECT system.bucket(2, CAST(null AS decimal))")); + assertThat(scalarSql("SELECT system.bucket(64, CAST('12.34' as DECIMAL(9, 2)))")).isEqualTo(56); + assertThat(scalarSql("SELECT system.bucket(18, CAST('12.30' as DECIMAL(9, 2)))")).isEqualTo(13); + assertThat(scalarSql("SELECT system.bucket(16, CAST('12.999' as DECIMAL(9, 3)))")).isEqualTo(2); + assertThat(scalarSql("SELECT system.bucket(32, CAST('0.05' as DECIMAL(5, 2)))")).isEqualTo(21); + assertThat(scalarSql("SELECT system.bucket(128, CAST('0.05' as DECIMAL(9, 2)))")).isEqualTo(85); + assertThat(scalarSql("SELECT system.bucket(18, CAST('0.05' as DECIMAL(9, 2)))")).isEqualTo(3); + + assertThat(scalarSql("SELECT system.bucket(2, CAST(null AS decimal))")) + .as("Null input should return null") + .isNull(); } - @Test + @TestTemplate public void testBucketTimestamp() { - Assert.assertEquals( - 99, scalarSql("SELECT system.bucket(100, TIMESTAMP '1997-01-01 00:00:00 UTC+00:00')")); - Assert.assertEquals( - 85, scalarSql("SELECT system.bucket(100, TIMESTAMP '1997-01-31 09:26:56 UTC+00:00')")); - Assert.assertEquals( - 62, scalarSql("SELECT system.bucket(100, TIMESTAMP '2022-08-08 00:00:00 UTC+00:00')")); - Assert.assertNull(scalarSql("SELECT system.bucket(2, CAST(null AS timestamp))")); + assertThat(scalarSql("SELECT system.bucket(100, TIMESTAMP '1997-01-01 00:00:00 UTC+00:00')")) + .isEqualTo(99); + assertThat(scalarSql("SELECT system.bucket(100, TIMESTAMP '1997-01-31 09:26:56 UTC+00:00')")) + .isEqualTo(85); + assertThat(scalarSql("SELECT system.bucket(100, TIMESTAMP '2022-08-08 00:00:00 UTC+00:00')")) + .isEqualTo(62); + assertThat(scalarSql("SELECT system.bucket(2, CAST(null AS timestamp))")).isNull(); } - @Test + @TestTemplate public void testBucketString() { - Assert.assertEquals(4, scalarSql("SELECT system.bucket(5, 'abcdefg')")); - Assert.assertEquals(122, scalarSql("SELECT system.bucket(128, 'abc')")); - Assert.assertEquals(54, scalarSql("SELECT system.bucket(64, 'abcde')")); - Assert.assertEquals(8, scalarSql("SELECT system.bucket(12, '测试')")); - Assert.assertEquals(1, scalarSql("SELECT system.bucket(16, '测试raul试测')")); - Assert.assertEquals( - "Varchar should work like string", - 1, - scalarSql("SELECT system.bucket(16, CAST('测试raul试测' AS varchar(8)))")); - Assert.assertEquals( - "Char should work like string", - 1, - scalarSql("SELECT system.bucket(16, CAST('测试raul试测' AS char(8)))")); - Assert.assertEquals( - "Should not fail on the empty string", 0, scalarSql("SELECT system.bucket(16, '')")); - Assert.assertNull( - "Null input should return null as output", - scalarSql("SELECT system.bucket(16, CAST(null AS string))")); + assertThat(scalarSql("SELECT system.bucket(5, 'abcdefg')")).isEqualTo(4); + assertThat(scalarSql("SELECT system.bucket(128, 'abc')")).isEqualTo(122); + assertThat(scalarSql("SELECT system.bucket(64, 'abcde')")).isEqualTo(54); + assertThat(scalarSql("SELECT system.bucket(12, '测试')")).isEqualTo(8); + assertThat(scalarSql("SELECT system.bucket(16, '测试raul试测')")).isEqualTo(1); + assertThat(scalarSql("SELECT system.bucket(16, CAST('测试raul试测' AS varchar(8)))")) + .as("Varchar should work like string") + .isEqualTo(1); + assertThat(scalarSql("SELECT system.bucket(16, CAST('测试raul试测' AS char(8)))")) + .as("Char should work like string") + .isEqualTo(1); + assertThat(scalarSql("SELECT system.bucket(16, '')")) + .as("Should not fail on the empty string") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.bucket(16, CAST(null AS string))")) + .as("Null input should return null as output") + .isNull(); } - @Test + @TestTemplate public void testBucketBinary() { - Assert.assertEquals( - 1, scalarSql("SELECT system.bucket(10, X'0102030405060708090a0b0c0d0e0f')")); - Assert.assertEquals(10, scalarSql("SELECT system.bucket(12, %s)", asBytesLiteral("abcdefg"))); - Assert.assertEquals(13, scalarSql("SELECT system.bucket(18, %s)", asBytesLiteral("abc\0\0"))); - Assert.assertEquals(42, scalarSql("SELECT system.bucket(48, %s)", asBytesLiteral("abc"))); - Assert.assertEquals(3, scalarSql("SELECT system.bucket(16, %s)", asBytesLiteral("测试_"))); - - Assert.assertNull( - "Null input should return null as output", - scalarSql("SELECT system.bucket(100, CAST(null AS binary))")); + assertThat(scalarSql("SELECT system.bucket(10, X'0102030405060708090a0b0c0d0e0f')")) + .isEqualTo(1); + assertThat(scalarSql("SELECT system.bucket(12, %s)", asBytesLiteral("abcdefg"))).isEqualTo(10); + assertThat(scalarSql("SELECT system.bucket(18, %s)", asBytesLiteral("abc\0\0"))).isEqualTo(13); + assertThat(scalarSql("SELECT system.bucket(48, %s)", asBytesLiteral("abc"))).isEqualTo(42); + assertThat(scalarSql("SELECT system.bucket(16, %s)", asBytesLiteral("测试_"))).isEqualTo(3); + + assertThat(scalarSql("SELECT system.bucket(100, CAST(null AS binary))")) + .as("Null input should return null as output") + .isNull(); } - @Test + @TestTemplate public void testNumBucketsAcceptsShortAndByte() { - Assert.assertEquals( - "Short types should be usable for the number of buckets field", - 1, - scalarSql("SELECT system.bucket(5S, 1L)")); - - Assert.assertEquals( - "Byte types should be allowed for the number of buckets field", - 1, - scalarSql("SELECT system.bucket(5Y, 1)")); + assertThat(scalarSql("SELECT system.bucket(5S, 1L)")) + .as("Short types should be usable for the number of buckets field") + .isEqualTo(1); + + assertThat(scalarSql("SELECT system.bucket(5Y, 1)")) + .as("Byte types should be allowed for the number of buckets field") + .isEqualTo(1); } - @Test + @TestTemplate public void testWrongNumberOfArguments() { Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket()")) .isInstanceOf(AnalysisException.class) @@ -213,7 +203,7 @@ public void testWrongNumberOfArguments() { "Function 'bucket' cannot process input: (int, bigint, int): Wrong number of inputs (expected numBuckets and value)"); } - @Test + @TestTemplate public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { Assertions.assertThatThrownBy( () -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")) @@ -245,7 +235,7 @@ public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { "Function 'bucket' cannot process input: (interval day to second, int): Expected number of buckets to be tinyint, shortint or int"); } - @Test + @TestTemplate public void testInvalidTypesForBucketColumn() { Assertions.assertThatThrownBy( () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")) @@ -285,7 +275,7 @@ public void testInvalidTypesForBucketColumn() { "Function 'bucket' cannot process input: (int, interval day to second)"); } - @Test + @TestTemplate public void testThatMagicFunctionsAreInvoked() { // TinyInt Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6Y)")) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java index 2c744e574fb9..36cf196351b8 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java @@ -18,93 +18,85 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.sql.Date; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; -public class TestSparkDaysFunction extends SparkTestBaseWithCatalog { +public class TestSparkDaysFunction extends TestBaseWithCatalog { - @Before + @BeforeEach public void useCatalog() { sql("USE %s", catalogName); } - @Test + @TestTemplate public void testDates() { - Assert.assertEquals( - "Expected to produce 2017-12-01", - Date.valueOf("2017-12-01"), - scalarSql("SELECT system.days(date('2017-12-01'))")); - Assert.assertEquals( - "Expected to produce 1970-01-01", - Date.valueOf("1970-01-01"), - scalarSql("SELECT system.days(date('1970-01-01'))")); - Assert.assertEquals( - "Expected to produce 1969-12-31", - Date.valueOf("1969-12-31"), - scalarSql("SELECT system.days(date('1969-12-31'))")); - Assert.assertNull(scalarSql("SELECT system.days(CAST(null AS DATE))")); + assertThat(scalarSql("SELECT system.days(date('2017-12-01'))")) + .as("Expected to produce 2017-12-01") + .isEqualTo(Date.valueOf("2017-12-01")); + assertThat(scalarSql("SELECT system.days(date('1970-01-01'))")) + .as("Expected to produce 1970-01-01") + .isEqualTo(Date.valueOf("1970-01-01")); + assertThat(scalarSql("SELECT system.days(date('1969-12-31'))")) + .as("Expected to produce 1969-12-31") + .isEqualTo(Date.valueOf("1969-12-31")); + assertThat(scalarSql("SELECT system.days(CAST(null AS DATE))")).isNull(); } - @Test + @TestTemplate public void testTimestamps() { - Assert.assertEquals( - "Expected to produce 2017-12-01", - Date.valueOf("2017-12-01"), - scalarSql("SELECT system.days(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")); - Assert.assertEquals( - "Expected to produce 1970-01-01", - Date.valueOf("1970-01-01"), - scalarSql("SELECT system.days(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")); - Assert.assertEquals( - "Expected to produce 1969-12-31", - Date.valueOf("1969-12-31"), - scalarSql("SELECT system.days(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")); - Assert.assertNull(scalarSql("SELECT system.days(CAST(null AS TIMESTAMP))")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")) + .as("Expected to produce 2017-12-01") + .isEqualTo(Date.valueOf("2017-12-01")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")) + .as("Expected to produce 1970-01-01") + .isEqualTo(Date.valueOf("1970-01-01")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")) + .as("Expected to produce 1969-12-31") + .isEqualTo(Date.valueOf("1969-12-31")); + assertThat(scalarSql("SELECT system.days(CAST(null AS TIMESTAMP))")).isNull(); } - @Test + @TestTemplate public void testTimestampNtz() { - Assert.assertEquals( - "Expected to produce 2017-12-01", - Date.valueOf("2017-12-01"), - scalarSql("SELECT system.days(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")); - Assert.assertEquals( - "Expected to produce 1970-01-01", - Date.valueOf("1970-01-01"), - scalarSql("SELECT system.days(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")); - Assert.assertEquals( - "Expected to produce 1969-12-31", - Date.valueOf("1969-12-31"), - scalarSql("SELECT system.days(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")); - Assert.assertNull(scalarSql("SELECT system.days(CAST(null AS TIMESTAMP_NTZ))")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")) + .as("Expected to produce 2017-12-01") + .isEqualTo(Date.valueOf("2017-12-01")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")) + .as("Expected to produce 1970-01-01") + .isEqualTo(Date.valueOf("1970-01-01")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")) + .as("Expected to produce 1969-12-31") + .isEqualTo(Date.valueOf("1969-12-31")); + assertThat(scalarSql("SELECT system.days(CAST(null AS TIMESTAMP_NTZ))")).isNull(); } - @Test + @TestTemplate public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days()")) + assertThatThrownBy(() -> scalarSql("SELECT system.days()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Function 'days' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.days(date('1969-12-31'), date('1969-12-31'))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'days' cannot process input: (date, date): Wrong number of inputs"); } - @Test + @TestTemplate public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.days(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'days' cannot process input: (int): Expected value to be date or timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.days(1L)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'days' cannot process input: (bigint): Expected value to be date or timestamp"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java index 5d839abc8871..17380747b4c0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java @@ -18,76 +18,71 @@ */ package org.apache.iceberg.spark.sql; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; -public class TestSparkHoursFunction extends SparkTestBaseWithCatalog { +public class TestSparkHoursFunction extends TestBaseWithCatalog { - @Before + @BeforeEach public void useCatalog() { sql("USE %s", catalogName); } - @Test + @TestTemplate public void testTimestamps() { - Assert.assertEquals( - "Expected to produce 17501 * 24 + 10", - 420034, - scalarSql("SELECT system.hours(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")); - Assert.assertEquals( - "Expected to produce 0 * 24 + 0 = 0", - 0, - scalarSql("SELECT system.hours(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")); - Assert.assertEquals( - "Expected to produce -1", - -1, - scalarSql("SELECT system.hours(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")); - Assert.assertNull(scalarSql("SELECT system.hours(CAST(null AS TIMESTAMP))")); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")) + .as("Expected to produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")) + .as("Expected to produce 0 * 24 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.hours(CAST(null AS TIMESTAMP))")).isNull(); } - @Test + @TestTemplate public void testTimestampsNtz() { - Assert.assertEquals( - "Expected to produce 17501 * 24 + 10", - 420034, - scalarSql("SELECT system.hours(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")); - Assert.assertEquals( - "Expected to produce 0 * 24 + 0 = 0", - 0, - scalarSql("SELECT system.hours(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")); - Assert.assertEquals( - "Expected to produce -1", - -1, - scalarSql("SELECT system.hours(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")); - Assert.assertNull(scalarSql("SELECT system.hours(CAST(null AS TIMESTAMP_NTZ))")); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")) + .as("Expected to produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")) + .as("Expected to produce 0 * 24 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.hours(CAST(null AS TIMESTAMP_NTZ))")).isNull(); } - @Test + @TestTemplate public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours()")) + assertThatThrownBy(() -> scalarSql("SELECT system.hours()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'hours' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.hours(date('1969-12-31'), date('1969-12-31'))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'hours' cannot process input: (date, date): Wrong number of inputs"); } - @Test + @TestTemplate public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.hours(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'hours' cannot process input: (int): Expected value to be timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.hours(1L)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'hours' cannot process input: (bigint): Expected value to be timestamp"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java index 9c90ec008d4c..1a00950124f0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java @@ -18,109 +18,103 @@ */ package org.apache.iceberg.spark.sql; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.spark.functions.MonthsFunction; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; -public class TestSparkMonthsFunction extends SparkTestBaseWithCatalog { +public class TestSparkMonthsFunction extends TestBaseWithCatalog { - @Before + @BeforeEach public void useCatalog() { sql("USE %s", catalogName); } - @Test + @TestTemplate public void testDates() { - Assert.assertEquals( - "Expected to produce 47 * 12 + 11 = 575", - 575, - scalarSql("SELECT system.months(date('2017-12-01'))")); - Assert.assertEquals( - "Expected to produce 0 * 12 + 0 = 0", - 0, - scalarSql("SELECT system.months(date('1970-01-01'))")); - Assert.assertEquals( - "Expected to produce -1", -1, scalarSql("SELECT system.months(date('1969-12-31'))")); - Assert.assertNull(scalarSql("SELECT system.months(CAST(null AS DATE))")); + assertThat(scalarSql("SELECT system.months(date('2017-12-01'))")) + .as("Expected to produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat(scalarSql("SELECT system.months(date('1970-01-01'))")) + .as("Expected to produce 0 * 12 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.months(date('1969-12-31'))")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.months(CAST(null AS DATE))")).isNull(); } - @Test + @TestTemplate public void testTimestamps() { - Assert.assertEquals( - "Expected to produce 47 * 12 + 11 = 575", - 575, - scalarSql("SELECT system.months(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")); - Assert.assertEquals( - "Expected to produce 0 * 12 + 0 = 0", - 0, - scalarSql("SELECT system.months(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")); - Assert.assertEquals( - "Expected to produce -1", - -1, - scalarSql("SELECT system.months(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")); - Assert.assertNull(scalarSql("SELECT system.months(CAST(null AS TIMESTAMP))")); + assertThat(scalarSql("SELECT system.months(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")) + .as("Expected to produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat(scalarSql("SELECT system.months(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")) + .as("Expected to produce 0 * 12 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.months(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.months(CAST(null AS TIMESTAMP))")).isNull(); } - @Test + @TestTemplate public void testTimestampNtz() { - Assert.assertEquals( - "Expected to produce 47 * 12 + 11 = 575", - 575, - scalarSql("SELECT system.months(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")); - Assert.assertEquals( - "Expected to produce 0 * 12 + 0 = 0", - 0, - scalarSql("SELECT system.months(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")); - Assert.assertEquals( - "Expected to produce -1", - -1, - scalarSql("SELECT system.months(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")); - Assert.assertNull(scalarSql("SELECT system.months(CAST(null AS TIMESTAMP_NTZ))")); + assertThat(scalarSql("SELECT system.months(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")) + .as("Expected to produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat(scalarSql("SELECT system.months(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")) + .as("Expected to produce 0 * 12 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.months(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.months(CAST(null AS TIMESTAMP_NTZ))")).isNull(); } - @Test + @TestTemplate public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months()")) + assertThatThrownBy(() -> scalarSql("SELECT system.months()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'months' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.months(date('1969-12-31'), date('1969-12-31'))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'months' cannot process input: (date, date): Wrong number of inputs"); } - @Test + @TestTemplate public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.months(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'months' cannot process input: (int): Expected value to be date or timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.months(1L)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'months' cannot process input: (bigint): Expected value to be date or timestamp"); } - @Test + @TestTemplate public void testThatMagicFunctionsAreInvoked() { String dateValue = "date('2017-12-01')"; String dateTransformClass = MonthsFunction.DateToMonthsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", dateValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", dateValue)) .asString() .isNotNull() .contains("staticinvoke(class " + dateTransformClass); String timestampValue = "TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00'"; String timestampTransformClass = MonthsFunction.TimestampToMonthsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", timestampValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", timestampValue)) .asString() .isNotNull() .contains("staticinvoke(class " + timestampTransformClass); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java index 7d3f360f8817..25f3770d01e4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java @@ -18,252 +18,234 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; -public class TestSparkTruncateFunction extends SparkTestBaseWithCatalog { - public TestSparkTruncateFunction() {} +public class TestSparkTruncateFunction extends TestBaseWithCatalog { - @Before + @BeforeEach public void useCatalog() { sql("USE %s", catalogName); } - @Test + @TestTemplate public void testTruncateTinyInt() { - Assert.assertEquals((byte) 0, scalarSql("SELECT system.truncate(10, 0Y)")); - Assert.assertEquals((byte) 0, scalarSql("SELECT system.truncate(10, 1Y)")); - Assert.assertEquals((byte) 0, scalarSql("SELECT system.truncate(10, 5Y)")); - Assert.assertEquals((byte) 0, scalarSql("SELECT system.truncate(10, 9Y)")); - Assert.assertEquals((byte) 10, scalarSql("SELECT system.truncate(10, 10Y)")); - Assert.assertEquals((byte) 10, scalarSql("SELECT system.truncate(10, 11Y)")); - Assert.assertEquals((byte) -10, scalarSql("SELECT system.truncate(10, -1Y)")); - Assert.assertEquals((byte) -10, scalarSql("SELECT system.truncate(10, -5Y)")); - Assert.assertEquals((byte) -10, scalarSql("SELECT system.truncate(10, -10Y)")); - Assert.assertEquals((byte) -20, scalarSql("SELECT system.truncate(10, -11Y)")); + assertThat(scalarSql("SELECT system.truncate(10, 0Y)")).isEqualTo((byte) 0); + assertThat(scalarSql("SELECT system.truncate(10, 1Y)")).isEqualTo((byte) 0); + assertThat(scalarSql("SELECT system.truncate(10, 5Y)")).isEqualTo((byte) 0); + assertThat(scalarSql("SELECT system.truncate(10, 9Y)")).isEqualTo((byte) 0); + assertThat(scalarSql("SELECT system.truncate(10, 10Y)")).isEqualTo((byte) 10); + assertThat(scalarSql("SELECT system.truncate(10, 11Y)")).isEqualTo((byte) 10); + assertThat(scalarSql("SELECT system.truncate(10, -1Y)")).isEqualTo((byte) -10); + assertThat(scalarSql("SELECT system.truncate(10, -5Y)")).isEqualTo((byte) -10); + assertThat(scalarSql("SELECT system.truncate(10, -10Y)")).isEqualTo((byte) -10); + assertThat(scalarSql("SELECT system.truncate(10, -11Y)")).isEqualTo((byte) -20); // Check that different widths can be used - Assert.assertEquals((byte) -2, scalarSql("SELECT system.truncate(2, -1Y)")); + assertThat(scalarSql("SELECT system.truncate(2, -1Y)")).isEqualTo((byte) -2); - Assert.assertNull( - "Null input should return null", - scalarSql("SELECT system.truncate(2, CAST(null AS tinyint))")); + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS tinyint))")) + .as("Null input should return null") + .isNull(); } - @Test + @TestTemplate public void testTruncateSmallInt() { - Assert.assertEquals((short) 0, scalarSql("SELECT system.truncate(10, 0S)")); - Assert.assertEquals((short) 0, scalarSql("SELECT system.truncate(10, 1S)")); - Assert.assertEquals((short) 0, scalarSql("SELECT system.truncate(10, 5S)")); - Assert.assertEquals((short) 0, scalarSql("SELECT system.truncate(10, 9S)")); - Assert.assertEquals((short) 10, scalarSql("SELECT system.truncate(10, 10S)")); - Assert.assertEquals((short) 10, scalarSql("SELECT system.truncate(10, 11S)")); - Assert.assertEquals((short) -10, scalarSql("SELECT system.truncate(10, -1S)")); - Assert.assertEquals((short) -10, scalarSql("SELECT system.truncate(10, -5S)")); - Assert.assertEquals((short) -10, scalarSql("SELECT system.truncate(10, -10S)")); - Assert.assertEquals((short) -20, scalarSql("SELECT system.truncate(10, -11S)")); + assertThat(scalarSql("SELECT system.truncate(10, 0S)")).isEqualTo((short) 0); + assertThat(scalarSql("SELECT system.truncate(10, 1S)")).isEqualTo((short) 0); + assertThat(scalarSql("SELECT system.truncate(10, 5S)")).isEqualTo((short) 0); + assertThat(scalarSql("SELECT system.truncate(10, 9S)")).isEqualTo((short) 0); + assertThat(scalarSql("SELECT system.truncate(10, 10S)")).isEqualTo((short) 10); + assertThat(scalarSql("SELECT system.truncate(10, 11S)")).isEqualTo((short) 10); + assertThat(scalarSql("SELECT system.truncate(10, -1S)")).isEqualTo((short) -10); + assertThat(scalarSql("SELECT system.truncate(10, -5S)")).isEqualTo((short) -10); + assertThat(scalarSql("SELECT system.truncate(10, -10S)")).isEqualTo((short) -10); + assertThat(scalarSql("SELECT system.truncate(10, -11S)")).isEqualTo((short) -20); // Check that different widths can be used - Assert.assertEquals((short) -2, scalarSql("SELECT system.truncate(2, -1S)")); + assertThat(scalarSql("SELECT system.truncate(2, -1S)")).isEqualTo((short) -2); - Assert.assertNull( - "Null input should return null", - scalarSql("SELECT system.truncate(2, CAST(null AS smallint))")); + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS smallint))")) + .as("Null input should return null") + .isNull(); } - @Test + @TestTemplate public void testTruncateInt() { - Assert.assertEquals(0, scalarSql("SELECT system.truncate(10, 0)")); - Assert.assertEquals(0, scalarSql("SELECT system.truncate(10, 1)")); - Assert.assertEquals(0, scalarSql("SELECT system.truncate(10, 5)")); - Assert.assertEquals(0, scalarSql("SELECT system.truncate(10, 9)")); - Assert.assertEquals(10, scalarSql("SELECT system.truncate(10, 10)")); - Assert.assertEquals(10, scalarSql("SELECT system.truncate(10, 11)")); - Assert.assertEquals(-10, scalarSql("SELECT system.truncate(10, -1)")); - Assert.assertEquals(-10, scalarSql("SELECT system.truncate(10, -5)")); - Assert.assertEquals(-10, scalarSql("SELECT system.truncate(10, -10)")); - Assert.assertEquals(-20, scalarSql("SELECT system.truncate(10, -11)")); + assertThat(scalarSql("SELECT system.truncate(10, 0)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.truncate(10, 1)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.truncate(10, 5)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.truncate(10, 9)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.truncate(10, 10)")).isEqualTo(10); + assertThat(scalarSql("SELECT system.truncate(10, 11)")).isEqualTo(10); + assertThat(scalarSql("SELECT system.truncate(10, -1)")).isEqualTo(-10); + assertThat(scalarSql("SELECT system.truncate(10, -5)")).isEqualTo(-10); + assertThat(scalarSql("SELECT system.truncate(10, -10)")).isEqualTo(-10); + assertThat(scalarSql("SELECT system.truncate(10, -11)")).isEqualTo(-20); // Check that different widths can be used - Assert.assertEquals(-2, scalarSql("SELECT system.truncate(2, -1)")); - Assert.assertEquals(0, scalarSql("SELECT system.truncate(300, 1)")); + assertThat(scalarSql("SELECT system.truncate(2, -1)")).isEqualTo(-2); + assertThat(scalarSql("SELECT system.truncate(300, 1)")).isEqualTo(0); - Assert.assertNull( - "Null input should return null", scalarSql("SELECT system.truncate(2, CAST(null AS int))")); + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS int))")) + .as("Null input should return null") + .isNull(); } - @Test + @TestTemplate public void testTruncateBigInt() { - Assert.assertEquals(0L, scalarSql("SELECT system.truncate(10, 0L)")); - Assert.assertEquals(0L, scalarSql("SELECT system.truncate(10, 1L)")); - Assert.assertEquals(0L, scalarSql("SELECT system.truncate(10, 5L)")); - Assert.assertEquals(0L, scalarSql("SELECT system.truncate(10, 9L)")); - Assert.assertEquals(10L, scalarSql("SELECT system.truncate(10, 10L)")); - Assert.assertEquals(10L, scalarSql("SELECT system.truncate(10, 11L)")); - Assert.assertEquals(-10L, scalarSql("SELECT system.truncate(10, -1L)")); - Assert.assertEquals(-10L, scalarSql("SELECT system.truncate(10, -5L)")); - Assert.assertEquals(-10L, scalarSql("SELECT system.truncate(10, -10L)")); - Assert.assertEquals(-20L, scalarSql("SELECT system.truncate(10, -11L)")); + assertThat(scalarSql("SELECT system.truncate(10, 0L)")).isEqualTo(0L); + assertThat(scalarSql("SELECT system.truncate(10, 1L)")).isEqualTo(0L); + assertThat(scalarSql("SELECT system.truncate(10, 5L)")).isEqualTo(0L); + assertThat(scalarSql("SELECT system.truncate(10, 9L)")).isEqualTo(0L); + assertThat(scalarSql("SELECT system.truncate(10, 10L)")).isEqualTo(10L); + assertThat(scalarSql("SELECT system.truncate(10, 11L)")).isEqualTo(10L); + assertThat(scalarSql("SELECT system.truncate(10, -1L)")).isEqualTo(-10L); + assertThat(scalarSql("SELECT system.truncate(10, -5L)")).isEqualTo(-10L); + assertThat(scalarSql("SELECT system.truncate(10, -10L)")).isEqualTo(-10L); + assertThat(scalarSql("SELECT system.truncate(10, -11L)")).isEqualTo(-20L); // Check that different widths can be used - Assert.assertEquals(-2L, scalarSql("SELECT system.truncate(2, -1L)")); + assertThat(scalarSql("SELECT system.truncate(2, -1L)")).isEqualTo(-2L); - Assert.assertNull( - "Null input should return null", - scalarSql("SELECT system.truncate(2, CAST(null AS bigint))")); + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS bigint))")) + .as("Null input should return null") + .isNull(); } - @Test + @TestTemplate public void testTruncateDecimal() { // decimal truncation works by applying the decimal scale to the width: ie 10 scale 2 = 0.10 - Assert.assertEquals( - new BigDecimal("12.30"), - scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "12.34")); + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "12.34")) + .isEqualTo(new BigDecimal("12.30")); - Assert.assertEquals( - new BigDecimal("12.30"), - scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "12.30")); + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "12.30")) + .isEqualTo(new BigDecimal("12.30")); - Assert.assertEquals( - new BigDecimal("12.290"), - scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 3)))", "12.299")); + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 3)))", "12.299")) + .isEqualTo(new BigDecimal("12.290")); - Assert.assertEquals( - new BigDecimal("0.03"), - scalarSql("SELECT system.truncate(3, CAST(%s as DECIMAL(5, 2)))", "0.05")); + assertThat(scalarSql("SELECT system.truncate(3, CAST(%s as DECIMAL(5, 2)))", "0.05")) + .isEqualTo(new BigDecimal("0.03")); - Assert.assertEquals( - new BigDecimal("0.00"), - scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "0.05")); + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "0.05")) + .isEqualTo(new BigDecimal("0.00")); - Assert.assertEquals( - new BigDecimal("-0.10"), - scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "-0.05")); + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "-0.05")) + .isEqualTo(new BigDecimal("-0.10")); - Assert.assertEquals( - "Implicit decimal scale and precision should be allowed", - new BigDecimal("12345.3480"), - scalarSql("SELECT system.truncate(10, 12345.3482)")); + assertThat(scalarSql("SELECT system.truncate(10, 12345.3482)")) + .as("Implicit decimal scale and precision should be allowed") + .isEqualTo(new BigDecimal("12345.3480")); BigDecimal truncatedDecimal = (BigDecimal) scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(6, 4)))", "-0.05"); - Assert.assertEquals( - "Truncating a decimal should return a decimal with the same scale", - 4, - truncatedDecimal.scale()); - - Assert.assertEquals( - "Truncating a decimal should return a decimal with the correct scale", - BigDecimal.valueOf(-500, 4), - truncatedDecimal); - - Assert.assertNull( - "Null input should return null", - scalarSql("SELECT system.truncate(2, CAST(null AS decimal))")); + assertThat(truncatedDecimal.scale()) + .as("Truncating a decimal should return a decimal with the same scale") + .isEqualTo(4); + + assertThat(truncatedDecimal) + .as("Truncating a decimal should return a decimal with the correct scale") + .isEqualTo(BigDecimal.valueOf(-500, 4)); + + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS decimal))")) + .as("Null input should return null") + .isNull(); } @SuppressWarnings("checkstyle:AvoidEscapedUnicodeCharacters") - @Test + @TestTemplate public void testTruncateString() { - Assert.assertEquals( - "Should system.truncate strings longer than length", - "abcde", - scalarSql("SELECT system.truncate(5, 'abcdefg')")); - - Assert.assertEquals( - "Should not pad strings shorter than length", - "abc", - scalarSql("SELECT system.truncate(5, 'abc')")); - - Assert.assertEquals( - "Should not alter strings equal to length", - "abcde", - scalarSql("SELECT system.truncate(5, 'abcde')")); - - Assert.assertEquals( - "Strings with multibyte unicode characters should should truncate along codepoint boundaries", - "イロ", - scalarSql("SELECT system.truncate(2, 'イロハニホヘト')")); - - Assert.assertEquals( - "Strings with multibyte unicode characters should truncate along codepoint boundaries", - "イロハ", - scalarSql("SELECT system.truncate(3, 'イロハニホヘト')")); - - Assert.assertEquals( - "Strings with multibyte unicode characters should not alter input with fewer codepoints than width", - "イロハニホヘト", - scalarSql("SELECT system.truncate(7, 'イロハニホヘト')")); + assertThat(scalarSql("SELECT system.truncate(5, 'abcdefg')")) + .as("Should system.truncate strings longer than length") + .isEqualTo("abcde"); + + assertThat(scalarSql("SELECT system.truncate(5, 'abc')")) + .as("Should not pad strings shorter than length") + .isEqualTo("abc"); + + assertThat(scalarSql("SELECT system.truncate(5, 'abcde')")) + .as("Should not alter strings equal to length") + .isEqualTo("abcde"); + + assertThat(scalarSql("SELECT system.truncate(2, 'イロハニホヘト')")) + .as("Strings with multibyte unicode characters should truncate along codepoint boundaries") + .isEqualTo("イロ"); + + assertThat(scalarSql("SELECT system.truncate(3, 'イロハニホヘト')")) + .as("Strings with multibyte unicode characters should truncate along codepoint boundaries") + .isEqualTo("イロハ"); + + assertThat(scalarSql("SELECT system.truncate(7, 'イロハニホヘト')")) + .as( + "Strings with multibyte unicode characters should not alter input with fewer codepoints than width") + .isEqualTo("イロハニホヘト"); String stringWithTwoCodePointsEachFourBytes = "\uD800\uDC00\uD800\uDC00"; - Assert.assertEquals( - "String truncation on four byte codepoints should work as expected", - "\uD800\uDC00", - scalarSql("SELECT system.truncate(1, '%s')", stringWithTwoCodePointsEachFourBytes)); - - Assert.assertEquals( - "Should handle three-byte UTF-8 characters appropriately", - "测", - scalarSql("SELECT system.truncate(1, '测试')")); - - Assert.assertEquals( - "Should handle three-byte UTF-8 characters mixed with two byte utf-8 characters", - "测试ra", - scalarSql("SELECT system.truncate(4, '测试raul试测')")); - - Assert.assertEquals( - "Should not fail on the empty string", "", scalarSql("SELECT system.truncate(10, '')")); - - Assert.assertNull( - "Null input should return null as output", - scalarSql("SELECT system.truncate(3, CAST(null AS string))")); - - Assert.assertEquals( - "Varchar should work like string", - "测试ra", - scalarSql("SELECT system.truncate(4, CAST('测试raul试测' AS varchar(8)))")); - - Assert.assertEquals( - "Char should work like string", - "测试ra", - scalarSql("SELECT system.truncate(4, CAST('测试raul试测' AS char(8)))")); + assertThat(scalarSql("SELECT system.truncate(1, '%s')", stringWithTwoCodePointsEachFourBytes)) + .as("String truncation on four byte codepoints should work as expected") + .isEqualTo("\uD800\uDC00"); + + assertThat(scalarSql("SELECT system.truncate(1, '测试')")) + .as("Should handle three-byte UTF-8 characters appropriately") + .isEqualTo("测"); + + assertThat(scalarSql("SELECT system.truncate(4, '测试raul试测')")) + .as("Should handle three-byte UTF-8 characters mixed with two byte utf-8 characters") + .isEqualTo("测试ra"); + + assertThat(scalarSql("SELECT system.truncate(10, '')")) + .as("Should not fail on the empty string") + .isEqualTo(""); + + assertThat(scalarSql("SELECT system.truncate(3, CAST(null AS string))")) + .as("Null input should return null as output") + .isNull(); + + assertThat(scalarSql("SELECT system.truncate(4, CAST('测试raul试测' AS varchar(8)))")) + .as("Varchar should work like string") + .isEqualTo("测试ra"); + + assertThat(scalarSql("SELECT system.truncate(4, CAST('测试raul试测' AS char(8)))")) + .as("Char should work like string") + .isEqualTo("测试ra"); } - @Test + @TestTemplate public void testTruncateBinary() { - Assert.assertArrayEquals( - new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, - (byte[]) scalarSql("SELECT system.truncate(10, X'0102030405060708090a0b0c0d0e0f')")); - Assert.assertArrayEquals( - "Should return the same input when value is equal to truncation width", - "abc".getBytes(StandardCharsets.UTF_8), - (byte[]) scalarSql("SELECT system.truncate(3, %s)", asBytesLiteral("abcdefg"))); - Assert.assertArrayEquals( - "Should not truncate, pad, or trim the input when its length is less than the width", - "abc\0\0".getBytes(StandardCharsets.UTF_8), - (byte[]) scalarSql("SELECT system.truncate(10, %s)", asBytesLiteral("abc\0\0"))); - Assert.assertArrayEquals( - "Should not pad the input when its length is equal to the width", - "abc".getBytes(StandardCharsets.UTF_8), - (byte[]) scalarSql("SELECT system.truncate(3, %s)", asBytesLiteral("abc"))); - Assert.assertArrayEquals( - "Should handle three-byte UTF-8 characters appropriately", - "测试".getBytes(StandardCharsets.UTF_8), - (byte[]) scalarSql("SELECT system.truncate(6, %s)", asBytesLiteral("测试_"))); - - Assert.assertNull( - "Null input should return null as output", - scalarSql("SELECT system.truncate(3, CAST(null AS binary))")); + assertThat((byte[]) scalarSql("SELECT system.truncate(10, X'0102030405060708090a0b0c0d0e0f')")) + .isEqualTo(new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}); + + assertThat((byte[]) scalarSql("SELECT system.truncate(3, %s)", asBytesLiteral("abcdefg"))) + .as("Should return the same input when value is equal to truncation width") + .isEqualTo("abc".getBytes(StandardCharsets.UTF_8)); + + assertThat((byte[]) scalarSql("SELECT system.truncate(10, %s)", asBytesLiteral("abc\0\0"))) + .as("Should not truncate, pad, or trim the input when its length is less than the width") + .isEqualTo("abc\0\0".getBytes(StandardCharsets.UTF_8)); + + assertThat((byte[]) scalarSql("SELECT system.truncate(3, %s)", asBytesLiteral("abc"))) + .as("Should not pad the input when its length is equal to the width") + .isEqualTo("abc".getBytes(StandardCharsets.UTF_8)); + + assertThat((byte[]) scalarSql("SELECT system.truncate(6, %s)", asBytesLiteral("测试_"))) + .as("Should handle three-byte UTF-8 characters appropriately") + .isEqualTo("测试".getBytes(StandardCharsets.UTF_8)); + + assertThat(scalarSql("SELECT system.truncate(3, CAST(null AS binary))")) + .as("Null input should return null as output") + .isNull(); } - @Test + @TestTemplate public void testTruncateUsingDataframeForWidthWithVaryingWidth() { // This situation is atypical but allowed. Typically, width is static as data is partitioned on // one width. @@ -276,63 +258,60 @@ public void testTruncateUsingDataframeForWidthWithVaryingWidth() { .selectExpr("system.truncate(width, value) as truncated_value") .filter("truncated_value == 0") .count(); - Assert.assertEquals( - "A truncate function with variable widths should be usable on dataframe columns", - rumRows, - numNonZero); + assertThat(numNonZero) + .as("A truncate function with variable widths should be usable on dataframe columns") + .isEqualTo(rumRows); } - @Test + @TestTemplate public void testWidthAcceptsShortAndByte() { - Assert.assertEquals( - "Short types should be usable for the width field", - 0L, - scalarSql("SELECT system.truncate(5S, 1L)")); - - Assert.assertEquals( - "Byte types should be allowed for the width field", - 0, - scalarSql("SELECT system.truncate(5Y, 1)")); + assertThat(scalarSql("SELECT system.truncate(5S, 1L)")) + .as("Short types should be usable for the width field") + .isEqualTo(0L); + + assertThat(scalarSql("SELECT system.truncate(5Y, 1)")) + .as("Byte types should be allowed for the width field") + .isEqualTo(0); } - @Test + @TestTemplate public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate()")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (): Wrong number of inputs (expected width and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int): Wrong number of inputs (expected width and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1, 1L, 1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1, 1L, 1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, bigint, int): Wrong number of inputs (expected width and value)"); } - @Test + @TestTemplate public void testInvalidTypesCannotBeUsedForWidth() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.truncate(CAST('12.34' as DECIMAL(9, 2)), 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (decimal(9,2), int): Expected truncation width to be tinyint, shortint or int"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate('5', 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate('5', 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (string, int): Expected truncation width to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.truncate(INTERVAL '100-00' YEAR TO MONTH, 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (interval year to month, int): Expected truncation width to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "SELECT system.truncate(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) @@ -341,42 +320,40 @@ public void testInvalidTypesCannotBeUsedForWidth() { "Function 'truncate' cannot process input: (interval day to second, int): Expected truncation width to be tinyint, shortint or int"); } - @Test + @TestTemplate public void testInvalidTypesForTruncationColumn() { - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as float))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, cast(12.3456 as float))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, float): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as double))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, cast(12.3456 as double))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, double): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, true)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, true)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, boolean): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, map(1, 1))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, map(1, 1))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, map): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, array(1L))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, array(1L))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, array): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.truncate(10, INTERVAL '100-00' YEAR TO MONTH)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, interval year to month): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "SELECT system.truncate(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) @@ -385,74 +362,73 @@ public void testInvalidTypesForTruncationColumn() { "Function 'truncate' cannot process input: (int, interval day to second): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); } - @Test + @TestTemplate public void testMagicFunctionsResolveForTinyIntAndSmallIntWidths() { // Magic functions have staticinvoke in the explain output. Nonmagic calls use // applyfunctionexpression instead. String tinyIntWidthExplain = (String) scalarSql("EXPLAIN EXTENDED SELECT system.truncate(1Y, 6)"); - Assertions.assertThat(tinyIntWidthExplain) + assertThat(tinyIntWidthExplain) .contains("cast(1 as int)") .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateInt"); String smallIntWidth = (String) scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5S, 6L)"); - Assertions.assertThat(smallIntWidth) + assertThat(smallIntWidth) .contains("cast(5 as int)") .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateBigInt"); } - @Test + @TestTemplate public void testThatMagicFunctionsAreInvoked() { // Magic functions have `staticinvoke` in the explain output. // Non-magic calls have `applyfunctionexpression` instead. // TinyInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6Y)")) + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6Y)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateTinyInt"); // SmallInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6S)")) + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6S)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateSmallInt"); // Int - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6)")) + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateInt"); // Long - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 6L)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 6L)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateBigInt"); // String - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 'abcdefg')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 'abcdefg')")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateString"); // Decimal - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 12.34)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 12.34)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateDecimal"); // Binary - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.truncate(4, X'0102030405060708')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(4, X'0102030405060708')")) .asString() .isNotNull() .contains( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java index 84c91e1d29ef..8cf62b2b48f3 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java @@ -18,111 +18,103 @@ */ package org.apache.iceberg.spark.sql; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.spark.functions.YearsFunction; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; -public class TestSparkYearsFunction extends SparkTestBaseWithCatalog { +public class TestSparkYearsFunction extends TestBaseWithCatalog { - @Before + @BeforeEach public void useCatalog() { sql("USE %s", catalogName); } - @Test + @TestTemplate public void testDates() { - Assert.assertEquals( - "Expected to produce 2017 - 1970 = 47", - 47, - scalarSql("SELECT system.years(date('2017-12-01'))")); - Assert.assertEquals( - "Expected to produce 1970 - 1970 = 0", - 0, - scalarSql("SELECT system.years(date('1970-01-01'))")); - Assert.assertEquals( - "Expected to produce 1969 - 1970 = -1", - -1, - scalarSql("SELECT system.years(date('1969-12-31'))")); - Assert.assertNull(scalarSql("SELECT system.years(CAST(null AS DATE))")); + assertThat(scalarSql("SELECT system.years(date('2017-12-01'))")) + .as("Expected to produce 2017 - 1970 = 47") + .isEqualTo(47); + assertThat(scalarSql("SELECT system.years(date('1970-01-01'))")) + .as("Expected to produce 1970 - 1970 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.years(date('1969-12-31'))")) + .as("Expected to produce 1969 - 1970 = -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.years(CAST(null AS DATE))")).isNull(); } - @Test + @TestTemplate public void testTimestamps() { - Assert.assertEquals( - "Expected to produce 2017 - 1970 = 47", - 47, - scalarSql("SELECT system.years(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")); - Assert.assertEquals( - "Expected to produce 1970 - 1970 = 0", - 0, - scalarSql("SELECT system.years(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")); - Assert.assertEquals( - "Expected to produce 1969 - 1970 = -1", - -1, - scalarSql("SELECT system.years(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")); - Assert.assertNull(scalarSql("SELECT system.years(CAST(null AS TIMESTAMP))")); + assertThat(scalarSql("SELECT system.years(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")) + .as("Expected to produce 2017 - 1970 = 47") + .isEqualTo(47); + assertThat(scalarSql("SELECT system.years(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")) + .as("Expected to produce 1970 - 1970 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.years(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")) + .as("Expected to produce 1969 - 1970 = -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.years(CAST(null AS TIMESTAMP))")).isNull(); } - @Test + @TestTemplate public void testTimestampNtz() { - Assert.assertEquals( - "Expected to produce 2017 - 1970 = 47", - 47, - scalarSql("SELECT system.years(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")); - Assert.assertEquals( - "Expected to produce 1970 - 1970 = 0", - 0, - scalarSql("SELECT system.years(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")); - Assert.assertEquals( - "Expected to produce 1969 - 1970 = -1", - -1, - scalarSql("SELECT system.years(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")); - Assert.assertNull(scalarSql("SELECT system.years(CAST(null AS TIMESTAMP_NTZ))")); + assertThat(scalarSql("SELECT system.years(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")) + .as("Expected to produce 2017 - 1970 = 47") + .isEqualTo(47); + assertThat(scalarSql("SELECT system.years(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")) + .as("Expected to produce 1970 - 1970 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.years(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")) + .as("Expected to produce 1969 - 1970 = -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.years(CAST(null AS TIMESTAMP_NTZ))")).isNull(); } - @Test + @TestTemplate public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years()")) + assertThatThrownBy(() -> scalarSql("SELECT system.years()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'years' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.years(date('1969-12-31'), date('1969-12-31'))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'years' cannot process input: (date, date): Wrong number of inputs"); } - @Test + @TestTemplate public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.years(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'years' cannot process input: (int): Expected value to be date or timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.years(1L)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'years' cannot process input: (bigint): Expected value to be date or timestamp"); } - @Test + @TestTemplate public void testThatMagicFunctionsAreInvoked() { String dateValue = "date('2017-12-01')"; String dateTransformClass = YearsFunction.DateToYearsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", dateValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", dateValue)) .asString() .isNotNull() .contains("staticinvoke(class " + dateTransformClass); String timestampValue = "TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00'"; String timestampTransformClass = YearsFunction.TimestampToYearsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", timestampValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", timestampValue)) .asString() .isNotNull() .contains("staticinvoke(class " + timestampTransformClass); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java index 8db3f6e821b6..6719c45ca961 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java @@ -20,11 +20,15 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.lang3.StringUtils; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -32,10 +36,11 @@ import org.apache.iceberg.expressions.Expressions; 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.SparkSQLProperties; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.spark.data.RandomData; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; @@ -44,19 +49,30 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; -import org.junit.After; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog { - - @Parameterized.Parameters(name = "planningMode = {0}") - public static Object[] parameters() { - return new Object[] {LOCAL, DISTRIBUTED}; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestStoragePartitionedJoins extends TestBaseWithCatalog { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, planningMode = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + LOCAL + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + DISTRIBUTED + }, + }; } private static final String OTHER_TABLE_NAME = "other_table"; @@ -96,18 +112,15 @@ public static Object[] parameters() { SparkSQLProperties.PRESERVE_DATA_GROUPING, "true"); - private final PlanningMode planningMode; - - public TestStoragePartitionedJoins(PlanningMode planningMode) { - this.planningMode = planningMode; - } + @Parameter(index = 3) + private PlanningMode planningMode; - @BeforeClass + @BeforeAll public static void setupSparkConf() { spark.conf().set("spark.sql.shuffle.partitions", "4"); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME)); @@ -115,107 +128,107 @@ public void removeTables() { // TODO: add tests for truncate transforms once SPARK-40295 is released - @Test + @TestTemplate public void testJoinsWithBucketingOnByteColumn() throws NoSuchTableException { checkJoin("byte_col", "TINYINT", "bucket(4, byte_col)"); } - @Test + @TestTemplate public void testJoinsWithBucketingOnShortColumn() throws NoSuchTableException { checkJoin("short_col", "SMALLINT", "bucket(4, short_col)"); } - @Test + @TestTemplate public void testJoinsWithBucketingOnIntColumn() throws NoSuchTableException { checkJoin("int_col", "INT", "bucket(16, int_col)"); } - @Test + @TestTemplate public void testJoinsWithBucketingOnLongColumn() throws NoSuchTableException { checkJoin("long_col", "BIGINT", "bucket(16, long_col)"); } - @Test + @TestTemplate public void testJoinsWithBucketingOnTimestampColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP", "bucket(16, timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithBucketingOnTimestampNtzColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP_NTZ", "bucket(16, timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithBucketingOnDateColumn() throws NoSuchTableException { checkJoin("date_col", "DATE", "bucket(8, date_col)"); } - @Test + @TestTemplate public void testJoinsWithBucketingOnDecimalColumn() throws NoSuchTableException { checkJoin("decimal_col", "DECIMAL(20, 2)", "bucket(8, decimal_col)"); } - @Test + @TestTemplate public void testJoinsWithBucketingOnBinaryColumn() throws NoSuchTableException { checkJoin("binary_col", "BINARY", "bucket(8, binary_col)"); } - @Test + @TestTemplate public void testJoinsWithYearsOnTimestampColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP", "years(timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithYearsOnTimestampNtzColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP_NTZ", "years(timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithYearsOnDateColumn() throws NoSuchTableException { checkJoin("date_col", "DATE", "years(date_col)"); } - @Test + @TestTemplate public void testJoinsWithMonthsOnTimestampColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP", "months(timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithMonthsOnTimestampNtzColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP_NTZ", "months(timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithMonthsOnDateColumn() throws NoSuchTableException { checkJoin("date_col", "DATE", "months(date_col)"); } - @Test + @TestTemplate public void testJoinsWithDaysOnTimestampColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP", "days(timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithDaysOnTimestampNtzColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP_NTZ", "days(timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithDaysOnDateColumn() throws NoSuchTableException { checkJoin("date_col", "DATE", "days(date_col)"); } - @Test + @TestTemplate public void testJoinsWithHoursOnTimestampColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP", "hours(timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithHoursOnTimestampNtzColumn() throws NoSuchTableException { checkJoin("timestamp_col", "TIMESTAMP_NTZ", "hours(timestamp_col)"); } - @Test + @TestTemplate public void testJoinsWithMultipleTransformTypes() throws NoSuchTableException { String createTableStmt = "CREATE TABLE %s (" @@ -304,7 +317,7 @@ public void testJoinsWithMultipleTransformTypes() throws NoSuchTableException { tableName(OTHER_TABLE_NAME)); } - @Test + @TestTemplate public void testJoinsWithCompatibleSpecEvolution() { // create a table with an empty spec sql( @@ -357,7 +370,7 @@ public void testJoinsWithCompatibleSpecEvolution() { tableName(OTHER_TABLE_NAME)); } - @Test + @TestTemplate public void testJoinsWithIncompatibleSpecs() { sql( "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" @@ -397,7 +410,7 @@ public void testJoinsWithIncompatibleSpecs() { tableName(OTHER_TABLE_NAME)); } - @Test + @TestTemplate public void testJoinsWithUnpartitionedTables() { sql( "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" @@ -437,7 +450,7 @@ public void testJoinsWithUnpartitionedTables() { tableName(OTHER_TABLE_NAME)); } - @Test + @TestTemplate public void testJoinsWithEmptyTable() { sql( "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" @@ -469,7 +482,7 @@ public void testJoinsWithEmptyTable() { tableName(OTHER_TABLE_NAME)); } - @Test + @TestTemplate public void testJoinsWithOneSplitTables() { sql( "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" @@ -501,7 +514,7 @@ public void testJoinsWithOneSplitTables() { tableName(OTHER_TABLE_NAME)); } - @Test + @TestTemplate public void testJoinsWithMismatchingPartitionKeys() { sql( "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" @@ -535,7 +548,7 @@ public void testJoinsWithMismatchingPartitionKeys() { tableName(OTHER_TABLE_NAME)); } - @Test + @TestTemplate public void testAggregates() throws NoSuchTableException { sql( "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" @@ -626,10 +639,9 @@ private void assertPartitioningAwarePlan( () -> { String plan = executeAndKeepPlan(query, args).toString(); int actualNumShuffles = StringUtils.countMatches(plan, "Exchange"); - Assert.assertEquals( - "Number of shuffles with enabled SPJ must match", - expectedNumShufflesWithSPJ, - actualNumShuffles); + assertThat(actualNumShuffles) + .as("Number of shuffles with enabled SPJ must match") + .isEqualTo(expectedNumShufflesWithSPJ); rowsWithSPJ.set(sql(query, args)); }); @@ -639,10 +651,9 @@ private void assertPartitioningAwarePlan( () -> { String plan = executeAndKeepPlan(query, args).toString(); int actualNumShuffles = StringUtils.countMatches(plan, "Exchange"); - Assert.assertEquals( - "Number of shuffles with disabled SPJ must match", - expectedNumShufflesWithoutSPJ, - actualNumShuffles); + assertThat(actualNumShuffles) + .as("Number of shuffles with disabled SPJ must match") + .isEqualTo(expectedNumShufflesWithoutSPJ); rowsWithoutSPJ.set(sql(query, args)); }); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java index 3c85209c1bef..636e789e16fa 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java @@ -18,35 +18,33 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.sql.Timestamp; import java.time.LocalDateTime; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkSessionCatalog; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.joda.time.DateTime; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; -public class TestTimestampWithoutZone extends SparkCatalogTestBase { +public class TestTimestampWithoutZone extends CatalogTestBase { private static final String newTableName = "created_table"; - private final Map config; - private static final Schema schema = new Schema( Types.NestedField.required(1, "id", Types.LongType.get()), @@ -59,7 +57,7 @@ public class TestTimestampWithoutZone extends SparkCatalogTestBase { row(2L, toLocalDateTime("2021-01-01T00:00:00.0"), toTimestamp("2021-02-01T00:00:00.0")), row(3L, toLocalDateTime("2021-01-01T00:00:00.0"), toTimestamp("2021-02-01T00:00:00.0"))); - @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { return new Object[][] { { @@ -74,18 +72,12 @@ public static Object[][] parameters() { }; } - public TestTimestampWithoutZone( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - this.config = config; - } - - @Before + @BeforeEach public void createTables() { validationCatalog.createTable(tableIdent, schema); } - @After + @AfterEach public void removeTables() { validationCatalog.dropTable(tableIdent, true); sql("DROP TABLE IF EXISTS %s", newTableName); @@ -117,7 +109,7 @@ public void removeTables() { 8 */ - @Test + @TestTemplate public void testAppendTimestampWithoutZone() { // Both NTZ sql( @@ -131,7 +123,7 @@ public void testAppendTimestampWithoutZone() { toLocalDateTime("2021-02-01T00:00:00.0"))))); } - @Test + @TestTemplate public void testAppendTimestampWithZone() { // Both TZ sql( @@ -145,16 +137,15 @@ public void testAppendTimestampWithZone() { toTimestamp("2021-02-01T00:00:00.0"))))); } - @Test + @TestTemplate public void testCreateAsSelectWithTimestampWithoutZone() { sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); - Assert.assertEquals( - "Should have " + values.size() + " row", - (long) values.size(), - scalarSql("SELECT count(*) FROM %s", newTableName)); + assertThat(scalarSql("SELECT count(*) FROM %s", newTableName)) + .as("Should have " + values.size() + " row") + .isEqualTo((long) values.size()); assertEquals( "Row data should match expected", @@ -162,16 +153,15 @@ public void testCreateAsSelectWithTimestampWithoutZone() { sql("SELECT * FROM %s ORDER BY id", newTableName)); } - @Test + @TestTemplate public void testCreateNewTableShouldHaveTimestampWithZoneIcebergType() { sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); - Assert.assertEquals( - "Should have " + values.size() + " row", - (long) values.size(), - scalarSql("SELECT count(*) FROM %s", newTableName)); + assertThat(scalarSql("SELECT count(*) FROM %s", newTableName)) + .as("Should have " + values.size() + " row") + .isEqualTo((long) values.size()); assertEquals( "Data from created table should match data from base table", @@ -183,21 +173,20 @@ public void testCreateNewTableShouldHaveTimestampWithZoneIcebergType() { assertFieldsType(createdTable.schema(), Types.TimestampType.withZone(), "tsz"); } - @Test + @TestTemplate public void testCreateNewTableShouldHaveTimestampWithoutZoneIcebergType() { spark .sessionState() .catalogManager() .currentCatalog() - .initialize(catalog.name(), new CaseInsensitiveStringMap(config)); + .initialize(catalog.name(), new CaseInsensitiveStringMap(catalogConfig)); sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); - Assert.assertEquals( - "Should have " + values.size() + " row", - (long) values.size(), - scalarSql("SELECT count(*) FROM %s", newTableName)); + assertThat(scalarSql("SELECT count(*) FROM %s", newTableName)) + .as("Should have " + values.size() + " row") + .isEqualTo((long) values.size()); assertEquals( "Row data should match expected", @@ -246,6 +235,6 @@ private void assertFieldsType(Schema actual, Type.PrimitiveType expected, String .select(fields) .asStruct() .fields() - .forEach(field -> Assert.assertEquals(expected, field.type())); + .forEach(field -> assertThat(field.type()).isEqualTo(expected)); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java index d01ccab00f55..7d9dfe95efc0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java @@ -18,12 +18,4 @@ */ package org.apache.iceberg.spark.sql; -import java.util.Map; - -public class TestUnpartitionedWrites extends UnpartitionedWritesTestBase { - - public TestUnpartitionedWrites( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } -} +public class TestUnpartitionedWrites extends UnpartitionedWritesTestBase {} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java index 1f5bee42af05..a0c664b03b83 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java @@ -18,21 +18,16 @@ */ package org.apache.iceberg.spark.sql; -import java.util.Map; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; public class TestUnpartitionedWritesToBranch extends UnpartitionedWritesTestBase { private static final String BRANCH = "test"; - public TestUnpartitionedWritesToBranch( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - @Override public void createTables() { super.createTables(); @@ -51,9 +46,9 @@ protected String selectTarget() { return String.format("%s VERSION AS OF '%s'", tableName, BRANCH); } - @Test + @TestTemplate public void testInsertIntoNonExistingBranchFails() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("INSERT INTO %s.branch_not_exist VALUES (4, 'd'), (5, 'e')", tableName)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot use branch (does not exist): not_exist"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java index 58918249d83f..2d97dfec186d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java @@ -18,50 +18,46 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.util.List; -import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.source.SimpleRecord; 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.functions; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; - -public abstract class UnpartitionedWritesTestBase extends SparkCatalogTestBase { - public UnpartitionedWritesTestBase( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public abstract class UnpartitionedWritesTestBase extends CatalogTestBase { - @Before + @BeforeEach public void createTables() { sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testInsertAppend() { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); sql("INSERT INTO %s VALUES (4, 'd'), (5, 'e')", commitTarget()); - Assert.assertEquals( - "Should have 5 rows after insert", - 5L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows") + .isEqualTo(5L); List expected = ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e")); @@ -72,17 +68,17 @@ public void testInsertAppend() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testInsertOverwrite() { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); sql("INSERT OVERWRITE %s VALUES (4, 'd'), (5, 'e')", commitTarget()); - Assert.assertEquals( - "Should have 2 rows after overwrite", - 2L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 2 rows after overwrite") + .isEqualTo(2L); List expected = ImmutableList.of(row(4L, "d"), row(5L, "e")); @@ -92,9 +88,9 @@ public void testInsertOverwrite() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testInsertAppendAtSnapshot() { - Assume.assumeTrue(tableName.equals(commitTarget())); + assumeThat(tableName.equals(commitTarget())).isTrue(); long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; @@ -105,9 +101,9 @@ public void testInsertAppendAtSnapshot() { .hasMessageStartingWith("Cannot write to table at a specific snapshot"); } - @Test + @TestTemplate public void testInsertOverwriteAtSnapshot() { - Assume.assumeTrue(tableName.equals(commitTarget())); + assumeThat(tableName.equals(commitTarget())).isTrue(); long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; @@ -120,20 +116,20 @@ public void testInsertOverwriteAtSnapshot() { .hasMessageStartingWith("Cannot write to table at a specific snapshot"); } - @Test + @TestTemplate public void testDataFrameV2Append() throws NoSuchTableException { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); ds.writeTo(commitTarget()).append(); - Assert.assertEquals( - "Should have 5 rows after insert", - 5L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows after insert") + .isEqualTo(5L); List expected = ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e")); @@ -144,20 +140,20 @@ public void testDataFrameV2Append() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); ds.writeTo(commitTarget()).overwritePartitions(); - Assert.assertEquals( - "Should have 2 rows after overwrite", - 2L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 2 rows after overwrite") + .isEqualTo(2L); List expected = ImmutableList.of(row(4L, "d"), row(5L, "e")); @@ -167,20 +163,20 @@ public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDataFrameV2Overwrite() throws NoSuchTableException { - Assert.assertEquals( - "Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); ds.writeTo(commitTarget()).overwrite(functions.col("id").$less$eq(3)); - Assert.assertEquals( - "Should have 2 rows after overwrite", - 2L, - scalarSql("SELECT count(*) FROM %s", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 2 rows after overwrite") + .isEqualTo(2L); List expected = ImmutableList.of(row(4L, "d"), row(5L, "e"));