This is an automated email from the ASF dual-hosted git repository.

etudenhoefner pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/main by this push:
     new b3d5133343 Spark: Migrate Spark 3.4 test base to JUnit5 (#12501)
b3d5133343 is described below

commit b3d5133343ff6b15b7c0201e662b545533d5eba7
Author: Tom Tanaka <[email protected]>
AuthorDate: Fri Mar 14 17:09:52 2025 +0900

    Spark: Migrate Spark 3.4 test base to JUnit5 (#12501)
---
 .../org/apache/iceberg/spark/CatalogTestBase.java  |  59 +++++++
 .../java/org/apache/iceberg/spark/TestBase.java    |   6 +-
 .../apache/iceberg/spark/TestBaseWithCatalog.java  | 195 +++++++++++++++++++++
 .../apache/iceberg/spark/TestFunctionCatalog.java  |  69 ++++----
 .../iceberg/spark/source/TestPathIdentifier.java   |  30 ++--
 .../apache/iceberg/spark/sql/TestAlterTable.java   | 155 ++++++++--------
 .../java/org/apache/iceberg/spark/TestBase.java    |   4 +-
 .../iceberg/spark/source/TestPathIdentifier.java   |   1 -
 .../apache/iceberg/spark/sql/TestAlterTable.java   |   8 +-
 9 files changed, 383 insertions(+), 144 deletions(-)

diff --git 
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java
new file mode 100644
index 0000000000..87a49b6444
--- /dev/null
+++ 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.ParameterizedTestExtension;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+@ExtendWith(ParameterizedTestExtension.class)
+public abstract class CatalogTestBase extends TestBaseWithCatalog {
+
+  // these parameters are broken out to avoid changes that need to modify lots 
of test suites
+  @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HIVE.catalogName(),
+        SparkCatalogConfig.HIVE.implementation(),
+        SparkCatalogConfig.HIVE.properties()
+      },
+      {
+        SparkCatalogConfig.HADOOP.catalogName(),
+        SparkCatalogConfig.HADOOP.implementation(),
+        SparkCatalogConfig.HADOOP.properties()
+      },
+      {
+        SparkCatalogConfig.SPARK.catalogName(),
+        SparkCatalogConfig.SPARK.implementation(),
+        SparkCatalogConfig.SPARK.properties()
+      },
+      {
+        SparkCatalogConfig.REST.catalogName(),
+        SparkCatalogConfig.REST.implementation(),
+        ImmutableMap.builder()
+            .putAll(SparkCatalogConfig.REST.properties())
+            .put(CatalogProperties.URI, 
restCatalog.properties().get(CatalogProperties.URI))
+            .build()
+      }
+    };
+  }
+}
diff --git 
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
similarity index 98%
copy from spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
copy to spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
index de68351f6e..2d327519e0 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
+++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
@@ -118,7 +118,7 @@ public abstract class TestBase extends SparkTestHelperBase {
 
   protected List<Object[]> sql(String query, Object... args) {
     List<Row> rows = spark.sql(String.format(query, args)).collectAsList();
-    if (rows.isEmpty()) {
+    if (rows.size() < 1) {
       return ImmutableList.of();
     }
 
@@ -127,9 +127,9 @@ public abstract class TestBase extends SparkTestHelperBase {
 
   protected Object scalarSql(String query, Object... args) {
     List<Object[]> rows = sql(query, args);
-    assertThat(rows.size()).as("Scalar SQL should return one 
row").isEqualTo(1);
+    assertThat(rows).as("Scalar SQL should return one row").hasSize(1);
     Object[] row = Iterables.getOnlyElement(rows);
-    assertThat(row.length).as("Scalar SQL should return one 
value").isEqualTo(1);
+    assertThat(row).as("Scalar SQL should return one value").hasSize(1);
     return row[0];
   }
 
diff --git 
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java
 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java
new file mode 100644
index 0000000000..8d3e65ce7c
--- /dev/null
+++ 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import static org.apache.iceberg.CatalogProperties.CATALOG_IMPL;
+import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE;
+import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP;
+import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE;
+import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Parameter;
+import org.apache.iceberg.ParameterizedTestExtension;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.PlanningMode;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.inmemory.InMemoryCatalog;
+import org.apache.iceberg.rest.RESTCatalog;
+import org.apache.iceberg.rest.RESTCatalogServer;
+import org.apache.iceberg.rest.RESTServerExtension;
+import org.apache.iceberg.util.PropertyUtil;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+@ExtendWith(ParameterizedTestExtension.class)
+public abstract class TestBaseWithCatalog extends TestBase {
+  protected static File warehouse = null;
+
+  @RegisterExtension
+  private static final RESTServerExtension REST_SERVER_EXTENSION =
+      new RESTServerExtension(
+          Map.of(
+              RESTCatalogServer.REST_PORT,
+              RESTServerExtension.FREE_PORT,
+              // In-memory sqlite database by default is private to the 
connection that created it.
+              // If more than 1 jdbc connection backed by in-memory sqlite is 
created behind one
+              // JdbcCatalog, then different jdbc connections could provide 
different views of table
+              // status even belonging to the same catalog. Reference:
+              // https://www.sqlite.org/inmemorydb.html
+              CatalogProperties.CLIENT_POOL_SIZE,
+              "1"));
+
+  protected static RESTCatalog restCatalog;
+
+  @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  protected static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HADOOP.catalogName(),
+        SparkCatalogConfig.HADOOP.implementation(),
+        SparkCatalogConfig.HADOOP.properties()
+      },
+    };
+  }
+
+  @BeforeAll
+  public static void setUpAll() throws IOException {
+    TestBaseWithCatalog.warehouse = File.createTempFile("warehouse", null);
+    assertThat(warehouse.delete()).isTrue();
+    restCatalog = REST_SERVER_EXTENSION.client();
+  }
+
+  @AfterAll
+  public static void tearDownAll() throws IOException {
+    if (warehouse != null && warehouse.exists()) {
+      Path warehousePath = new Path(warehouse.getAbsolutePath());
+      FileSystem fs = warehousePath.getFileSystem(hiveConf);
+      assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + 
warehousePath).isTrue();
+    }
+  }
+
+  @TempDir protected java.nio.file.Path temp;
+
+  @Parameter(index = 0)
+  protected String catalogName;
+
+  @Parameter(index = 1)
+  protected String implementation;
+
+  @Parameter(index = 2)
+  protected Map<String, String> catalogConfig;
+
+  protected Catalog validationCatalog;
+  protected SupportsNamespaces validationNamespaceCatalog;
+  protected TableIdentifier tableIdent = 
TableIdentifier.of(Namespace.of("default"), "table");
+  protected String tableName;
+
+  @BeforeEach
+  public void before() {
+    configureValidationCatalog();
+
+    spark.conf().set("spark.sql.catalog." + catalogName, implementation);
+    catalogConfig.forEach(
+        (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + 
"." + key, value));
+
+    if ("hadoop".equalsIgnoreCase(catalogConfig.get("type"))) {
+      spark.conf().set("spark.sql.catalog." + catalogName + ".warehouse", 
"file:" + warehouse);
+    }
+
+    this.tableName =
+        (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + 
"default.table";
+
+    sql("CREATE NAMESPACE IF NOT EXISTS default");
+  }
+
+  protected String tableName(String name) {
+    return (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + 
"default." + name;
+  }
+
+  protected String commitTarget() {
+    return tableName;
+  }
+
+  protected String selectTarget() {
+    return tableName;
+  }
+
+  protected boolean cachingCatalogEnabled() {
+    return PropertyUtil.propertyAsBoolean(
+        catalogConfig, CatalogProperties.CACHE_ENABLED, 
CatalogProperties.CACHE_ENABLED_DEFAULT);
+  }
+
+  protected void configurePlanningMode(PlanningMode planningMode) {
+    configurePlanningMode(tableName, planningMode);
+  }
+
+  protected void configurePlanningMode(String table, PlanningMode 
planningMode) {
+    sql(
+        "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')",
+        table,
+        TableProperties.DATA_PLANNING_MODE,
+        planningMode.modeName(),
+        TableProperties.DELETE_PLANNING_MODE,
+        planningMode.modeName());
+  }
+
+  private void configureValidationCatalog() {
+    if (catalogConfig.containsKey(ICEBERG_CATALOG_TYPE)) {
+      switch (catalogConfig.get(ICEBERG_CATALOG_TYPE)) {
+        case ICEBERG_CATALOG_TYPE_HADOOP:
+          this.validationCatalog =
+              new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" 
+ warehouse);
+          break;
+        case ICEBERG_CATALOG_TYPE_REST:
+          this.validationCatalog = restCatalog;
+          break;
+        case ICEBERG_CATALOG_TYPE_HIVE:
+          this.validationCatalog = catalog;
+          break;
+        default:
+          throw new IllegalArgumentException("Unknown catalog type");
+      }
+    } else if (catalogConfig.containsKey(CATALOG_IMPL)) {
+      switch (catalogConfig.get(CATALOG_IMPL)) {
+        case "org.apache.iceberg.inmemory.InMemoryCatalog":
+          this.validationCatalog = new InMemoryCatalog();
+          break;
+        default:
+          throw new IllegalArgumentException("Unknown catalog impl");
+      }
+    }
+    this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog;
+  }
+}
diff --git 
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java
 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java
index b51861e4a2..a146d5b702 100644
--- 
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java
+++ 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java
@@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.ParameterizedTestExtension;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
 import org.apache.iceberg.spark.functions.IcebergVersionFunction;
 import org.apache.spark.sql.AnalysisException;
@@ -30,33 +31,32 @@ import 
org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
 import org.apache.spark.sql.connector.catalog.FunctionCatalog;
 import org.apache.spark.sql.connector.catalog.Identifier;
 import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
 
-public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+@ExtendWith(ParameterizedTestExtension.class)
+public class TestFunctionCatalog extends TestBaseWithCatalog {
   private static final String[] EMPTY_NAMESPACE = new String[] {};
   private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
   private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
   private static final String[] DB_NAMESPACE = new String[] {"db"};
-  private final FunctionCatalog asFunctionCatalog;
+  private FunctionCatalog asFunctionCatalog;
 
-  public TestFunctionCatalog() {
-    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
-  }
-
-  @Before
+  @BeforeEach
   public void createDefaultNamespace() {
+    super.before();
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
     sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
   }
 
-  @After
+  @AfterEach
   public void dropDefaultNamespace() {
     sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
   }
 
-  @Test
+  @TestTemplate
   public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
     assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
         .anyMatch(func -> "iceberg_version".equals(func.name()));
@@ -64,17 +64,16 @@ public class TestFunctionCatalog extends 
SparkTestBaseWithCatalog {
     assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
         .anyMatch(func -> "iceberg_version".equals(func.name()));
 
-    Assert.assertArrayEquals(
-        "Listing functions in an existing namespace that's not system should 
not throw",
-        new Identifier[0],
-        asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE));
+    assertThat(asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE))
+        .as("Listing functions in an existing namespace that's not system 
should not throw")
+        .isEqualTo(new Identifier[0]);
 
     assertThatThrownBy(() -> asFunctionCatalog.listFunctions(DB_NAMESPACE))
         .isInstanceOf(NoSuchNamespaceException.class)
         .hasMessageStartingWith("[SCHEMA_NOT_FOUND] The schema `db` cannot be 
found.");
   }
 
-  @Test
+  @TestTemplate
   public void testLoadFunctions() throws NoSuchFunctionException {
     for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, 
SYSTEM_NAMESPACE)) {
       Identifier identifier = Identifier.of(namespace, "iceberg_version");
@@ -105,31 +104,31 @@ public class TestFunctionCatalog extends 
SparkTestBaseWithCatalog {
             "[UNRESOLVED_ROUTINE] Cannot resolve function `undefined_function` 
on search path");
   }
 
-  @Test
+  @TestTemplate
   public void testCallingFunctionInSQLEndToEnd() {
     String buildVersion = IcebergBuild.version();
 
-    Assert.assertEquals(
-        "Should be able to use the Iceberg version function from the fully 
qualified system namespace",
-        buildVersion,
-        scalarSql("SELECT %s.system.iceberg_version()", catalogName));
+    assertThat(scalarSql("SELECT %s.system.iceberg_version()", catalogName))
+        .as(
+            "Should be able to use the Iceberg version function from the fully 
qualified system namespace")
+        .isEqualTo(buildVersion);
 
-    Assert.assertEquals(
-        "Should be able to use the Iceberg version function when fully 
qualified without specifying a namespace",
-        buildVersion,
-        scalarSql("SELECT %s.iceberg_version()", catalogName));
+    assertThat(scalarSql("SELECT %s.iceberg_version()", catalogName))
+        .as(
+            "Should be able to use the Iceberg version function when fully 
qualified without specifying a namespace")
+        .isEqualTo(buildVersion);
 
     sql("USE %s", catalogName);
 
-    Assert.assertEquals(
-        "Should be able to call iceberg_version from system namespace without 
fully qualified name when using Iceberg catalog",
-        buildVersion,
-        scalarSql("SELECT system.iceberg_version()"));
+    assertThat(scalarSql("SELECT system.iceberg_version()"))
+        .as(
+            "Should be able to call iceberg_version from system namespace 
without fully qualified name when using Iceberg catalog")
+        .isEqualTo(buildVersion);
 
-    Assert.assertEquals(
-        "Should be able to call iceberg_version from empty namespace without 
fully qualified name when using Iceberg catalog",
-        buildVersion,
-        scalarSql("SELECT iceberg_version()"));
+    assertThat(scalarSql("SELECT iceberg_version()"))
+        .as(
+            "Should be able to call iceberg_version from empty namespace 
without fully qualified name when using Iceberg catalog")
+        .isEqualTo(buildVersion);
   }
 
   private FunctionCatalog castToFunctionCatalog(String name) {
diff --git 
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java
 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java
index e18c4d3241..aa8fe047b3 100644
--- 
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java
+++ 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java
@@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 import org.apache.iceberg.BaseTable;
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.hadoop.HadoopTableOperations;
@@ -30,41 +31,38 @@ import 
org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.spark.PathIdentifier;
 import org.apache.iceberg.spark.SparkCatalog;
 import org.apache.iceberg.spark.SparkSchemaUtil;
-import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.TestBase;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
 import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException;
 import org.apache.spark.sql.connector.expressions.Transform;
 import org.apache.spark.sql.util.CaseInsensitiveStringMap;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
 
-public class TestPathIdentifier extends SparkTestBase {
+public class TestPathIdentifier extends TestBase {
 
   private static final Schema SCHEMA =
       new Schema(
           required(1, "id", Types.LongType.get()), required(2, "data", 
Types.StringType.get()));
 
-  @Rule public TemporaryFolder temp = new TemporaryFolder();
+  @TempDir private Path temp;
   private File tableLocation;
   private PathIdentifier identifier;
   private SparkCatalog sparkCatalog;
 
-  @Before
+  @BeforeEach
   public void before() throws IOException {
-    tableLocation = temp.newFolder();
+    tableLocation = temp.toFile();
     identifier = new PathIdentifier(tableLocation.getAbsolutePath());
     sparkCatalog = new SparkCatalog();
     sparkCatalog.initialize("test", new 
CaseInsensitiveStringMap(ImmutableMap.of()));
   }
 
-  @After
+  @AfterEach
   public void after() {
-    tableLocation.delete();
     sparkCatalog = null;
   }
 
@@ -75,11 +73,11 @@ public class TestPathIdentifier extends SparkTestBase {
             sparkCatalog.createTable(
                 identifier, SparkSchemaUtil.convert(SCHEMA), new Transform[0], 
ImmutableMap.of());
 
-    Assert.assertEquals(table.table().location(), 
tableLocation.getAbsolutePath());
+    
assertThat(tableLocation.getAbsolutePath()).isEqualTo(table.table().location());
     assertThat(table.table()).isInstanceOf(BaseTable.class);
     assertThat(((BaseTable) 
table.table()).operations()).isInstanceOf(HadoopTableOperations.class);
 
-    Assert.assertEquals(sparkCatalog.loadTable(identifier), table);
-    Assert.assertTrue(sparkCatalog.dropTable(identifier));
+    assertThat(table).isEqualTo(sparkCatalog.loadTable(identifier));
+    assertThat(sparkCatalog.dropTable(identifier)).isTrue();
   }
 }
diff --git 
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java
 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java
index e5c6fcd0aa..f70df75d91 100644
--- 
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java
+++ 
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java
@@ -24,40 +24,37 @@ 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.ParameterizedTestExtension;
 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.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
 
-public class TestAlterTable extends SparkCatalogTestBase {
+@ExtendWith(ParameterizedTestExtension.class)
+public class TestAlterTable extends CatalogTestBase {
   private final TableIdentifier renamedIdent =
       TableIdentifier.of(Namespace.of("default"), "table2");
 
-  public TestAlterTable(String catalogName, String implementation, Map<String, 
String> 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)
@@ -65,7 +62,7 @@ public class TestAlterTable extends SparkCatalogTestBase {
             "Unsupported table change: Incompatible change: cannot add 
required column: c3");
   }
 
-  @Test
+  @TestTemplate
   public void testAddColumn() {
     sql(
         "ALTER TABLE %s ADD COLUMN point struct<x: double NOT NULL, y: double 
NOT NULL> AFTER id",
@@ -82,10 +79,9 @@ public class TestAlterTable extends SparkCatalogTestBase {
                     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);
 
@@ -101,13 +97,12 @@ public class TestAlterTable extends SparkCatalogTestBase {
                     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<struct<a:INT,b:INT,c:int>>", 
tableName);
     // use the implicit column name 'element' to access member of array and 
add column d to struct.
@@ -126,13 +121,12 @@ public class TestAlterTable extends SparkCatalogTestBase {
                         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<x:INT>, 
struct<a:INT,b:INT>>", tableName);
     // use the implicit column name 'key' and 'value' to access member of map.
@@ -153,10 +147,9 @@ public class TestAlterTable extends SparkCatalogTestBase {
                         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))
@@ -164,20 +157,19 @@ public class TestAlterTable extends SparkCatalogTestBase {
         .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);
 
@@ -186,13 +178,12 @@ public class TestAlterTable extends SparkCatalogTestBase {
             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);
 
@@ -201,13 +192,12 @@ public class TestAlterTable extends SparkCatalogTestBase {
             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);
@@ -218,13 +208,12 @@ public class TestAlterTable extends SparkCatalogTestBase {
             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);
 
@@ -233,13 +222,12 @@ public class TestAlterTable extends SparkCatalogTestBase {
             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);
@@ -249,17 +237,16 @@ public class TestAlterTable extends SparkCatalogTestBase {
             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);
@@ -270,13 +257,12 @@ public class TestAlterTable extends SparkCatalogTestBase {
             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);
@@ -287,13 +273,12 @@ public class TestAlterTable extends SparkCatalogTestBase {
             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() {
     assumeThat(catalogConfig.get(ICEBERG_CATALOG_TYPE))
         .as(
@@ -303,28 +288,32 @@ public class TestAlterTable extends SparkCatalogTestBase {
         .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);
 
-    
assertThat(validationCatalog.loadTable(tableIdent).properties().get("prop"))
+    assertThat(validationCatalog.loadTable(tableIdent).properties())
         .as("Should have the new table property")
-        .isEqualTo("value");
+        .containsEntry("prop", "value");
 
     sql("ALTER TABLE %s UNSET TBLPROPERTIES ('prop')", tableName);
 
-    
assertThat(validationCatalog.loadTable(tableIdent).properties().get("prop"))
+    assertThat(validationCatalog.loadTable(tableIdent).properties())
         .as("Should not have the removed table property")
-        .isNull();
+        .doesNotContainKey("prop");
 
     String[] reservedProperties = new String[] {"sort-order", 
"identifier-fields"};
     for (String reservedProp : reservedProperties) {
diff --git 
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java 
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
index de68351f6e..3e9f3334ef 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
@@ -127,9 +127,9 @@ public abstract class TestBase extends SparkTestHelperBase {
 
   protected Object scalarSql(String query, Object... args) {
     List<Object[]> rows = sql(query, args);
-    assertThat(rows.size()).as("Scalar SQL should return one 
row").isEqualTo(1);
+    assertThat(rows).as("Scalar SQL should return one row").hasSize(1);
     Object[] row = Iterables.getOnlyElement(rows);
-    assertThat(row.length).as("Scalar SQL should return one 
value").isEqualTo(1);
+    assertThat(row).as("Scalar SQL should return one value").hasSize(1);
     return row[0];
   }
 
diff --git 
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java
 
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java
index bb026b2ab2..aa8fe047b3 100644
--- 
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java
+++ 
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java
@@ -63,7 +63,6 @@ public class TestPathIdentifier extends TestBase {
 
   @AfterEach
   public void after() {
-    tableLocation.delete();
     sparkCatalog = null;
   }
 
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 5abc72606f..a3515f4465 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
@@ -302,15 +302,15 @@ public class TestAlterTable extends CatalogTestBase {
   public void testSetTableProperties() {
     sql("ALTER TABLE %s SET TBLPROPERTIES ('prop'='value')", tableName);
 
-    
assertThat(validationCatalog.loadTable(tableIdent).properties().get("prop"))
+    assertThat(validationCatalog.loadTable(tableIdent).properties())
         .as("Should have the new table property")
-        .isEqualTo("value");
+        .containsEntry("prop", "value");
 
     sql("ALTER TABLE %s UNSET TBLPROPERTIES ('prop')", tableName);
 
-    
assertThat(validationCatalog.loadTable(tableIdent).properties().get("prop"))
+    assertThat(validationCatalog.loadTable(tableIdent).properties())
         .as("Should not have the removed table property")
-        .isNull();
+        .doesNotContainKey("prop");
 
     String[] reservedProperties = new String[] {"sort-order", 
"identifier-fields"};
     for (String reservedProp : reservedProperties) {


Reply via email to