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

lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git


The following commit(s) were added to refs/heads/master by this push:
     new a4a8db1dbf [spark] Add checker for PaimonSparkSessionExtensions (#5327)
a4a8db1dbf is described below

commit a4a8db1dbf22e1ae2760be5b94c86fd97d649424
Author: Zouxxyy <[email protected]>
AuthorDate: Sat Mar 22 20:42:01 2025 +0800

    [spark] Add checker for PaimonSparkSessionExtensions (#5327)
---
 docs/content/spark/auxiliary.md                    |  4 +--
 .../generated/spark_connector_configuration.html   |  6 +++++
 .../java/org/apache/paimon/tests/SparkE2eTest.java |  2 ++
 .../paimon/spark/SparkGenericCatalogTest.java      |  4 ++-
 .../java/org/apache/paimon/spark/SparkCatalog.java |  3 +++
 .../apache/paimon/spark/SparkConnectorOptions.java |  7 +++++
 .../paimon/spark/commands/PaimonSparkWriter.scala  | 14 +---------
 .../org/apache/paimon/spark/util/OptionUtils.scala | 26 +++++++++++++++++++
 .../paimon/spark/SparkCatalogWithHiveTest.java     |  9 +++++++
 .../paimon/spark/SparkCatalogWithRestTest.java     |  3 +++
 .../paimon/spark/SparkGenericCatalogTest.java      | 16 ++++++++++--
 .../spark/SparkGenericCatalogWithHiveTest.java     |  9 +++++++
 .../org/apache/paimon/spark/SparkS3ITCase.java     | 12 ++++++---
 .../apache/paimon/spark/sql/PaimonOptionTest.scala | 28 +++++++++++++++++++-
 .../apache/paimon/spark/sql/SparkWriteITCase.scala | 12 +++++++--
 .../sql/SparkWriteWithNoExtensionITCase.scala      | 30 ----------------------
 16 files changed, 130 insertions(+), 55 deletions(-)

diff --git a/docs/content/spark/auxiliary.md b/docs/content/spark/auxiliary.md
index 2b316d9d5e..3739c85a92 100644
--- a/docs/content/spark/auxiliary.md
+++ b/docs/content/spark/auxiliary.md
@@ -29,6 +29,7 @@ under the License.
 ## Set / Reset
 The SET command sets a property, returns the value of an existing property or 
returns all SQLConf properties with value and meaning.
 The RESET command resets runtime configurations specific to the current 
session which were set via the SET command to their default values.
+
 To set dynamic options globally, you need add the `spark.paimon.` prefix. You 
can also set dynamic table options at this format: 
 `spark.paimon.${catalogName}.${dbName}.${tableName}.${config_key}`. The 
catalogName/dbName/tableName can be `*`, which means matching all 
 the specific parts. Dynamic table options will override global options if 
there are conflicts.
@@ -43,9 +44,6 @@ SET spark.paimon.file.block-size=512M;
 -- reset conf
 RESET spark.paimon.file.block-size;
 
--- set catalog
-USE paimon;
-
 -- set scan.snapshot-id=1 for the table default.T in any catalogs
 SET spark.paimon.*.default.T.scan.snapshot-id=1;
 SELECT * FROM default.T;
diff --git 
a/docs/layouts/shortcodes/generated/spark_connector_configuration.html 
b/docs/layouts/shortcodes/generated/spark_connector_configuration.html
index 00ca2ba17d..7074cbc05b 100644
--- a/docs/layouts/shortcodes/generated/spark_connector_configuration.html
+++ b/docs/layouts/shortcodes/generated/spark_connector_configuration.html
@@ -62,6 +62,12 @@ under the License.
             <td>Long</td>
             <td>The minimum number of rows returned in a single batch, which 
used to create MinRowsReadLimit with read.stream.maxTriggerDelayMs 
together.</td>
         </tr>
+        <tr>
+            <td><h5>requiredSparkConfsCheck.enabled</h5></td>
+            <td style="word-wrap: break-word;">true</td>
+            <td>Boolean</td>
+            <td>Whether to verify SparkSession is initialized with required 
configurations.</td>
+        </tr>
         <tr>
             <td><h5>write.merge-schema</h5></td>
             <td style="word-wrap: break-word;">false</td>
diff --git 
a/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/SparkE2eTest.java 
b/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/SparkE2eTest.java
index 1606d68967..2f87456d7d 100644
--- a/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/SparkE2eTest.java
+++ b/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/SparkE2eTest.java
@@ -62,6 +62,8 @@ public class SparkE2eTest extends E2eReaderTestBase {
                                             "--master",
                                             "spark://spark-master:7077",
                                             "--conf",
+                                            
"spark.sql.extensions=org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions",
+                                            "--conf",
                                             
"spark.sql.catalog.paimon=org.apache.paimon.spark.SparkCatalog",
                                             "--conf",
                                             
"spark.sql.catalog.paimon.warehouse=file:"
diff --git 
a/paimon-spark/paimon-spark-3.2/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java
 
b/paimon-spark/paimon-spark-3.2/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java
index bb3e81c9dd..33c46dde2d 100644
--- 
a/paimon-spark/paimon-spark-3.2/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java
+++ 
b/paimon-spark/paimon-spark-3.2/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java
@@ -52,13 +52,15 @@ public class SparkGenericCatalogTest {
         warehousePath = new Path("file:" + tempDir.toString());
         spark =
                 SparkSession.builder()
+                        .config(
+                                "spark.sql.catalog.spark_catalog",
+                                SparkGenericCatalog.class.getName())
                         .config("spark.sql.warehouse.dir", 
warehousePath.toString())
                         .config(
                                 "spark.sql.extensions",
                                 PaimonSparkSessionExtensions.class.getName())
                         .master("local[2]")
                         .getOrCreate();
-        spark.conf().set("spark.sql.catalog.spark_catalog", 
SparkGenericCatalog.class.getName());
     }
 
     @AfterAll
diff --git 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java
 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java
index c51f369ec0..f6e29d7651 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java
+++ 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java
@@ -73,6 +73,7 @@ import static org.apache.paimon.CoreOptions.TYPE;
 import static org.apache.paimon.TableType.FORMAT_TABLE;
 import static org.apache.paimon.spark.SparkCatalogOptions.DEFAULT_DATABASE;
 import static org.apache.paimon.spark.SparkTypeUtils.toPaimonType;
+import static 
org.apache.paimon.spark.util.OptionUtils.checkRequiredConfigurations;
 import static org.apache.paimon.spark.util.OptionUtils.copyWithSQLConf;
 import static org.apache.paimon.spark.utils.CatalogUtils.checkNamespace;
 import static org.apache.paimon.spark.utils.CatalogUtils.removeCatalogName;
@@ -91,6 +92,8 @@ public class SparkCatalog extends SparkBaseCatalog implements 
SupportFunction, S
 
     @Override
     public void initialize(String name, CaseInsensitiveStringMap options) {
+        checkRequiredConfigurations();
+
         this.catalogName = name;
         CatalogContext catalogContext =
                 CatalogContext.create(
diff --git 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkConnectorOptions.java
 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkConnectorOptions.java
index 4ddbc6490f..539f5a11e3 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkConnectorOptions.java
+++ 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkConnectorOptions.java
@@ -25,6 +25,13 @@ import static org.apache.paimon.options.ConfigOptions.key;
 /** Options for spark connector. */
 public class SparkConnectorOptions {
 
+    public static final ConfigOption<Boolean> 
REQUIRED_SPARK_CONFS_CHECK_ENABLED =
+            key("requiredSparkConfsCheck.enabled")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription(
+                            "Whether to verify SparkSession is initialized 
with required configurations.");
+
     public static final ConfigOption<Boolean> MERGE_SCHEMA =
             key("write.merge-schema")
                     .booleanType()
diff --git 
a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala
 
b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala
index db9800b469..e974ad98b4 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala
+++ 
b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala
@@ -30,6 +30,7 @@ import org.apache.paimon.io.{CompactIncrement, DataIncrement, 
IndexIncrement}
 import org.apache.paimon.manifest.{FileKind, IndexManifestEntry}
 import org.apache.paimon.spark.{SparkInternalRowWrapper, SparkRow, 
SparkTableWrite, SparkTypeUtils}
 import org.apache.paimon.spark.schema.SparkSystemColumns.{BUCKET_COL, 
ROW_KIND_COL}
+import org.apache.paimon.spark.util.OptionUtils.paimonExtensionEnabled
 import org.apache.paimon.spark.util.SparkRowUtils
 import org.apache.paimon.table.BucketMode._
 import org.apache.paimon.table.FileStoreTable
@@ -60,11 +61,6 @@ case class PaimonSparkWriter(table: FileStoreTable) {
 
   private lazy val log = LoggerFactory.getLogger(classOf[PaimonSparkWriter])
 
-  private val extensionKey = "spark.sql.extensions"
-
-  private val paimonSparkExtension =
-    "org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions"
-
   @transient private lazy val serializer = new CommitMessageSerializer
 
   val writeBuilder: BatchWriteBuilder = table.newBatchWriteBuilder()
@@ -77,14 +73,6 @@ case class PaimonSparkWriter(table: FileStoreTable) {
     val sparkSession = data.sparkSession
     import sparkSession.implicits._
 
-    def paimonExtensionEnabled: Boolean = {
-      val extensions = 
sparkSession.sessionState.conf.getConfString(extensionKey)
-      if (extensions != null && extensions.contains(paimonSparkExtension)) {
-        true
-      } else {
-        false
-      }
-    }
     val withInitBucketCol = bucketMode match {
       case BUCKET_UNAWARE => data
       case CROSS_PARTITION if !data.schema.fieldNames.contains(ROW_KIND_COL) =>
diff --git 
a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/OptionUtils.scala
 
b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/OptionUtils.scala
index b60dd1fb21..d1688622d4 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/OptionUtils.scala
+++ 
b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/OptionUtils.scala
@@ -19,9 +19,12 @@
 package org.apache.paimon.spark.util
 
 import org.apache.paimon.catalog.Identifier
+import org.apache.paimon.options.ConfigOption
+import org.apache.paimon.spark.SparkConnectorOptions
 import org.apache.paimon.table.Table
 
 import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.internal.StaticSQLConf
 
 import java.util.{Map => JMap}
 import java.util.regex.Pattern
@@ -33,6 +36,29 @@ object OptionUtils extends SQLConfHelper {
   private val PAIMON_OPTION_PREFIX = "spark.paimon."
   private val SPARK_CATALOG_PREFIX = "spark.sql.catalog."
 
+  def paimonExtensionEnabled: Boolean = {
+    conf
+      .getConf(StaticSQLConf.SPARK_SESSION_EXTENSIONS)
+      .getOrElse(Seq.empty)
+      
.contains("org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
+  }
+
+  def getOptionString(option: ConfigOption[_]): String = {
+    conf.getConfString(s"$PAIMON_OPTION_PREFIX${option.key()}", 
option.defaultValue().toString)
+  }
+
+  def checkRequiredConfigurations(): Unit = {
+    if 
(getOptionString(SparkConnectorOptions.REQUIRED_SPARK_CONFS_CHECK_ENABLED).toBoolean)
 {
+      if (!paimonExtensionEnabled) {
+        throw new RuntimeException(
+          """
+            |When using Paimon, it is necessary to configure 
`spark.sql.extensions` and ensure that it includes 
`org.apache.paimon.spark.extensions.PaimonSparkSessionExtension`.
+            |You can disable this check by configuring 
`spark.paimon.requiredSparkConfsCheck.enabled` to `false`, but it is strongly 
discouraged to do so.
+            |""".stripMargin)
+      }
+    }
+  }
+
   def extractCatalogName(): Option[String] = {
     val sparkCatalogTemplate = String.format("%s([^.]*)$", 
SPARK_CATALOG_PREFIX)
     val sparkCatalogPattern = Pattern.compile(sparkCatalogTemplate)
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java
 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java
index 7623d1da8b..bb05e5224d 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java
+++ 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java
@@ -73,6 +73,9 @@ public class SparkCatalogWithHiveTest {
                         .config(
                                 "spark.sql.catalog.spark_catalog.warehouse",
                                 warehousePath.toString())
+                        .config(
+                                "spark.sql.extensions",
+                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
                         .master("local[2]")
                         .getOrCreate();
 
@@ -139,6 +142,9 @@ public class SparkCatalogWithHiveTest {
                         .config(
                                 "spark.sql.catalog.spark_catalog",
                                 SparkGenericCatalog.class.getName())
+                        .config(
+                                "spark.sql.extensions",
+                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
                         .master("local[2]")
                         .getOrCreate();
 
@@ -167,6 +173,9 @@ public class SparkCatalogWithHiveTest {
                         .config(
                                 "spark.sql.catalog.spark_catalog.warehouse",
                                 warehousePath.toString())
+                        .config(
+                                "spark.sql.extensions",
+                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
                         .master("local[2]")
                         .getOrCreate();
 
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithRestTest.java
 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithRestTest.java
index 0748cf9d08..4502ba0122 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithRestTest.java
+++ 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithRestTest.java
@@ -84,6 +84,9 @@ public class SparkCatalogWithRestTest {
                         .config(
                                 "spark.sql.catalog.paimon.token.provider",
                                 AuthProviderEnum.BEAR.identifier())
+                        .config(
+                                "spark.sql.extensions",
+                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
                         .master("local[2]")
                         .getOrCreate();
 
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java
 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java
index 8e7492b6c5..0ae0e91306 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java
+++ 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java
@@ -19,6 +19,7 @@
 package org.apache.paimon.spark;
 
 import org.apache.paimon.fs.Path;
+import org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions;
 
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SparkSession;
@@ -44,10 +45,15 @@ public class SparkGenericCatalogTest {
         warehousePath = new Path("file:" + tempDir.toString());
         spark =
                 SparkSession.builder()
+                        .config(
+                                "spark.sql.catalog.spark_catalog",
+                                SparkGenericCatalog.class.getName())
+                        .config(
+                                "spark.sql.extensions",
+                                PaimonSparkSessionExtensions.class.getName())
                         .config("spark.sql.warehouse.dir", 
warehousePath.toString())
                         .master("local[2]")
                         .getOrCreate();
-        spark.conf().set("spark.sql.catalog.spark_catalog", 
SparkGenericCatalog.class.getName());
     }
 
     @AfterEach
@@ -82,10 +88,16 @@ public class SparkGenericCatalogTest {
 
         spark =
                 SparkSession.builder()
+                        .config(
+                                "spark.sql.catalog.spark_catalog",
+                                SparkGenericCatalog.class.getName())
+                        .config(
+                                "spark.sql.extensions",
+                                PaimonSparkSessionExtensions.class.getName())
                         .config("spark.sql.warehouse.dir", 
warehousePath.toString())
                         .master("local[2]")
                         .getOrCreate();
-        spark.conf().set("spark.sql.catalog.spark_catalog", 
SparkGenericCatalog.class.getName());
+
         assertThatCode(
                         () ->
                                 spark.sql(
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java
 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java
index f3d4ba8789..604e2ea279 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java
+++ 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java
@@ -59,6 +59,9 @@ public class SparkGenericCatalogWithHiveTest {
                         .config(
                                 "spark.sql.catalog.spark_catalog",
                                 SparkGenericCatalog.class.getName())
+                        .config(
+                                "spark.sql.extensions",
+                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
                         .master("local[2]")
                         .getOrCreate();
 
@@ -85,6 +88,9 @@ public class SparkGenericCatalogWithHiveTest {
                         .config("spark.sql.catalog.paimon.warehouse", 
warehousePath.toString())
                         .config("spark.sql.catalogImplementation", "in-memory")
                         .config("spark.sql.catalog.paimon", 
SparkCatalog.class.getName())
+                        .config(
+                                "spark.sql.extensions",
+                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
                         .master("local[2]")
                         .getOrCreate();
         spark2.sql("USE paimon");
@@ -110,6 +116,9 @@ public class SparkGenericCatalogWithHiveTest {
                         .config(
                                 "spark.sql.catalog.spark_catalog",
                                 SparkGenericCatalog.class.getName())
+                        .config(
+                                "spark.sql.extensions",
+                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
                         .master("local[2]")
                         .getOrCreate();
 
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java
 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java
index 22c9f7175c..1f34ed0d5c 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java
+++ 
b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java
@@ -59,9 +59,15 @@ public class SparkS3ITCase {
     public static void startMetastoreAndSpark() {
         String path = MINIO_CONTAINER.getS3UriForDefaultBucket() + "/" + 
UUID.randomUUID();
         warehousePath = new Path(path);
-        spark = SparkSession.builder().master("local[2]").getOrCreate();
-        spark.conf().set("spark.sql.catalog.paimon", 
SparkCatalog.class.getName());
-        spark.conf().set("spark.sql.catalog.paimon.warehouse", 
warehousePath.toString());
+        spark =
+                SparkSession.builder()
+                        .master("local[2]")
+                        .config("spark.sql.catalog.paimon", 
SparkCatalog.class.getName())
+                        .config("spark.sql.catalog.paimon.warehouse", 
warehousePath.toString())
+                        .config(
+                                "spark.sql.extensions",
+                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
+                        .getOrCreate();
         MINIO_CONTAINER
                 .getS3ConfigOptions()
                 .forEach((k, v) -> 
spark.conf().set("spark.sql.catalog.paimon." + k, v));
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala
 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala
index 44df3e54ca..9f2b7cad7f 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala
+++ 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala
@@ -21,7 +21,9 @@ package org.apache.paimon.spark.sql
 import org.apache.paimon.spark.PaimonSparkTestBase
 import org.apache.paimon.table.FileStoreTableFactory
 
-import org.apache.spark.sql.Row
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.paimon.Utils
 import org.junit.jupiter.api.Assertions
 
 class PaimonOptionTest extends PaimonSparkTestBase {
@@ -203,3 +205,27 @@ class PaimonOptionTest extends PaimonSparkTestBase {
     }
   }
 }
+
+class PaimonConfigCheckTest extends SparkFunSuite {
+
+  test("Paimon Option: required confs check") {
+    for (checkConf <- Seq(true, false)) {
+      val spark = SparkSession
+        .builder()
+        .master("local[2]")
+        .config("spark.sql.catalog.paimon", 
"org.apache.paimon.spark.SparkCatalog")
+        .config("spark.sql.catalog.paimon.warehouse", 
Utils.createTempDir.getCanonicalPath)
+        .config("spark.paimon.requiredSparkConfsCheck.enabled", 
checkConf.toString)
+        .getOrCreate()
+      try {
+        if (checkConf) {
+          assertThrows[RuntimeException](spark.sql("USE paimon"))
+        } else {
+          spark.sql("USE paimon")
+        }
+      } finally {
+        spark.close()
+      }
+    }
+  }
+}
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkWriteITCase.scala
 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkWriteITCase.scala
index 96abb1b840..e6c113d75d 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkWriteITCase.scala
+++ 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkWriteITCase.scala
@@ -20,15 +20,23 @@ package org.apache.paimon.spark.sql
 
 import org.apache.paimon.spark.PaimonSparkTestBase
 
+import org.apache.spark.SparkConf
 import org.apache.spark.sql.Row
 import org.junit.jupiter.api.Assertions
 
 import java.sql.Timestamp
 import java.time.LocalDateTime
 
-class SparkWriteITCase extends PaimonSparkTestBase {
+class SparkWriteWithNoExtensionITCase extends SparkWriteITCase {
 
-  import testImplicits._
+  /** Disable the spark extension. */
+  override protected def sparkConf: SparkConf = {
+    super.sparkConf.remove("spark.sql.extensions")
+    super.sparkConf.set("spark.paimon.requiredSparkConfsCheck.enabled", 
"false")
+  }
+}
+
+class SparkWriteITCase extends PaimonSparkTestBase {
 
   test("Paimon Write: AllTypes") {
     withTable("AllTypesTable") {
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkWriteWithNoExtensionITCase.scala
 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkWriteWithNoExtensionITCase.scala
deleted file mode 100644
index 9daa9f7d3b..0000000000
--- 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkWriteWithNoExtensionITCase.scala
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.paimon.spark.sql
-
-import org.apache.spark.SparkConf
-
-/** Test for spark writer with extension disabled. */
-class SparkWriteWithNoExtensionITCase extends SparkWriteITCase {
-
-  /** Disable the spark extension. */
-  override protected def sparkConf: SparkConf = {
-    super.sparkConf.remove("spark.sql.extensions")
-  }
-}

Reply via email to