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 d014bcac52 [spark] Throw exception when creating partitioned table via 
CTAS (#5684)
d014bcac52 is described below

commit d014bcac52fb4c96c109ef717d0dbb1c881ce0c4
Author: Zouxxyy <[email protected]>
AuthorDate: Tue Jun 3 10:25:15 2025 +0800

    [spark] Throw exception when creating partitioned table via CTAS (#5684)
---
 .../shim/PaimonCreateTableAsSelectStrategy.scala   | 15 ++++++++
 .../shim/PaimonCreateTableAsSelectStrategy.scala   | 15 ++++++++
 .../shim/PaimonCreateTableAsSelectStrategy.scala   | 15 ++++++++
 .../java/org/apache/paimon/spark/SparkCatalog.java |  9 +++--
 .../paimon/spark/catalog/FormatTableCatalog.java   | 31 +++++++++++++++
 .../paimon/spark/catalog/SparkBaseCatalog.java     |  4 +-
 .../org/apache/paimon/spark/SparkSource.scala      |  3 +-
 .../shim/PaimonCreateTableAsSelectStrategy.scala   | 15 ++++++++
 .../org/apache/paimon/spark/sql/DDLTestBase.scala  | 33 ----------------
 .../spark/sql/DDLWithHiveCatalogTestBase.scala     | 45 ++++++++++++++++++++++
 .../paimon/spark/sql/FormatTableTestBase.scala     | 26 ++++++++++---
 11 files changed, 166 insertions(+), 45 deletions(-)

diff --git 
a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
 
b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
index 63f3dd79f7..05370ddc90 100644
--- 
a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
+++ 
b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
@@ -19,6 +19,7 @@
 package org.apache.spark.sql.execution.shim
 
 import org.apache.paimon.CoreOptions
+import org.apache.paimon.spark.catalog.FormatTableCatalog
 
 import org.apache.spark.sql.{SparkSession, Strategy}
 import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, 
LogicalPlan}
@@ -42,6 +43,20 @@ case class PaimonCreateTableAsSelectStrategy(spark: 
SparkSession) extends Strate
             case (key, _) => coreOptionKeys.contains(key)
           }
           val newProps = CatalogV2Util.withDefaultOwnership(props) ++ 
coreOptions
+
+          val isPartitionedFormatTable = {
+            catalog match {
+              case catalog: FormatTableCatalog =>
+                catalog.isFormatTable(newProps.get("provider").orNull) && 
parts.nonEmpty
+              case _ => false
+            }
+          }
+
+          if (isPartitionedFormatTable) {
+            throw new UnsupportedOperationException(
+              "Using CTAS with partitioned format table is not supported yet.")
+          }
+
           CreateTableAsSelectExec(
             catalog,
             ident,
diff --git 
a/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
 
b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
index 5f225a3e15..4d4104d1ed 100644
--- 
a/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
+++ 
b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.shim
 
 import org.apache.paimon.CoreOptions
 import org.apache.paimon.spark.SparkCatalog
+import org.apache.paimon.spark.catalog.FormatTableCatalog
 
 import org.apache.spark.sql.{SparkSession, Strategy}
 import org.apache.spark.sql.catalyst.analysis.ResolvedDBObjectName
@@ -54,6 +55,20 @@ case class PaimonCreateTableAsSelectStrategy(spark: 
SparkSession)
             case (key, _) => coreOptionKeys.contains(key)
           }
           val newTableSpec = tableSpec.copy(properties = tableSpec.properties 
++ coreOptions)
+
+          val isPartitionedFormatTable = {
+            catalog match {
+              case catalog: FormatTableCatalog =>
+                catalog.isFormatTable(newTableSpec.provider.orNull) && 
parts.nonEmpty
+              case _ => false
+            }
+          }
+
+          if (isPartitionedFormatTable) {
+            throw new UnsupportedOperationException(
+              "Using CTAS with partitioned format table is not supported yet.")
+          }
+
           CreateTableAsSelectExec(
             catalog.asTableCatalog,
             ident.asIdentifier,
diff --git 
a/paimon-spark/paimon-spark-3.4/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
 
b/paimon-spark/paimon-spark-3.4/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
index fcf33bf8fb..d5424493eb 100644
--- 
a/paimon-spark/paimon-spark-3.4/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
+++ 
b/paimon-spark/paimon-spark-3.4/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.shim
 
 import org.apache.paimon.CoreOptions
 import org.apache.paimon.spark.SparkCatalog
+import org.apache.paimon.spark.catalog.FormatTableCatalog
 
 import org.apache.spark.sql.{SparkSession, Strategy}
 import org.apache.spark.sql.catalyst.analysis.ResolvedIdentifier
@@ -56,6 +57,20 @@ case class PaimonCreateTableAsSelectStrategy(spark: 
SparkSession)
             case (key, _) => coreOptionKeys.contains(key)
           }
           val newTableSpec = tableSpec.copy(properties = tableSpec.properties 
++ coreOptions)
+
+          val isPartitionedFormatTable = {
+            catalog match {
+              case catalog: FormatTableCatalog =>
+                catalog.isFormatTable(newTableSpec.provider.orNull) && 
parts.nonEmpty
+              case _ => false
+            }
+          }
+
+          if (isPartitionedFormatTable) {
+            throw new UnsupportedOperationException(
+              "Using CTAS with partitioned format table is not supported yet.")
+          }
+
           CreateTableAsSelectExec(
             catalog.asTableCatalog,
             ident,
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 40d07d906a..df03286fd0 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
@@ -28,6 +28,7 @@ import org.apache.paimon.function.FunctionDefinition;
 import org.apache.paimon.options.Options;
 import org.apache.paimon.schema.Schema;
 import org.apache.paimon.schema.SchemaChange;
+import org.apache.paimon.spark.catalog.FormatTableCatalog;
 import org.apache.paimon.spark.catalog.SparkBaseCatalog;
 import org.apache.paimon.spark.catalog.SupportView;
 import org.apache.paimon.spark.catalog.functions.PaimonFunctions;
@@ -90,11 +91,11 @@ import static 
org.apache.paimon.spark.utils.CatalogUtils.toIdentifier;
 
 /** Spark {@link TableCatalog} for paimon. */
 public class SparkCatalog extends SparkBaseCatalog
-        implements SupportView, FunctionCatalog, SupportsNamespaces {
-
-    public static final String FUNCTION_DEFINITION_NAME = "spark";
+        implements SupportView, FunctionCatalog, SupportsNamespaces, 
FormatTableCatalog {
 
     private static final Logger LOG = 
LoggerFactory.getLogger(SparkCatalog.class);
+
+    public static final String FUNCTION_DEFINITION_NAME = "spark";
     private static final String PRIMARY_KEY_IDENTIFIER = "primary-key";
 
     protected Catalog catalog = null;
@@ -400,7 +401,7 @@ public class SparkCatalog extends SparkBaseCatalog
             StructType schema, Transform[] partitions, Map<String, String> 
properties) {
         Map<String, String> normalizedProperties = new HashMap<>(properties);
         String provider = properties.get(TableCatalog.PROP_PROVIDER);
-        if (!usePaimon(provider) && 
SparkSource.FORMAT_NAMES().contains(provider.toLowerCase())) {
+        if (!usePaimon(provider) && isFormatTable(provider)) {
             normalizedProperties.put(TYPE.key(), FORMAT_TABLE.toString());
             normalizedProperties.put(FILE_FORMAT.key(), 
provider.toLowerCase());
         }
diff --git 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/FormatTableCatalog.java
 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/FormatTableCatalog.java
new file mode 100644
index 0000000000..551fc54f73
--- /dev/null
+++ 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/FormatTableCatalog.java
@@ -0,0 +1,31 @@
+/*
+ * 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.catalog;
+
+import org.apache.paimon.spark.SparkSource;
+
+import javax.annotation.Nullable;
+
+/** Catalog supports format table. */
+public interface FormatTableCatalog {
+
+    default boolean isFormatTable(@Nullable String provide) {
+        return provide != null && 
SparkSource.FORMAT_NAMES().contains(provide.toLowerCase());
+    }
+}
diff --git 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SparkBaseCatalog.java
 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SparkBaseCatalog.java
index 55670a594d..d3d173c903 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SparkBaseCatalog.java
+++ 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SparkBaseCatalog.java
@@ -29,6 +29,8 @@ import org.apache.spark.sql.connector.catalog.Identifier;
 import org.apache.spark.sql.connector.catalog.SupportsNamespaces;
 import org.apache.spark.sql.connector.catalog.TableCatalog;
 
+import javax.annotation.Nullable;
+
 /** Spark base catalog. */
 public abstract class SparkBaseCatalog
         implements TableCatalog, SupportsNamespaces, ProcedureCatalog, 
WithPaimonCatalog {
@@ -51,7 +53,7 @@ public abstract class SparkBaseCatalog
         throw new NoSuchProcedureException(identifier);
     }
 
-    public boolean usePaimon(String provider) {
+    public boolean usePaimon(@Nullable String provider) {
         return provider == null || 
SparkSource.NAME().equalsIgnoreCase(provider);
     }
 }
diff --git 
a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala
 
b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala
index 287290d5b2..cdcd2a1668 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala
+++ 
b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala
@@ -26,6 +26,7 @@ import org.apache.paimon.spark.commands.WriteIntoPaimonTable
 import org.apache.paimon.spark.sources.PaimonSink
 import org.apache.paimon.spark.util.OptionUtils.{extractCatalogName, 
mergeSQLConfWithIdentifier}
 import org.apache.paimon.table.{DataTable, FileStoreTable, 
FileStoreTableFactory}
+import org.apache.paimon.table.FormatTable.Format
 import org.apache.paimon.table.system.AuditLogTable
 
 import org.apache.spark.sql.{DataFrame, SaveMode => SparkSaveMode, 
SparkSession, SQLContext}
@@ -118,7 +119,7 @@ object SparkSource {
 
   val NAME = "paimon"
 
-  val FORMAT_NAMES: Seq[String] = Seq("csv", "orc", "parquet", "json")
+  val FORMAT_NAMES: Seq[String] = 
Format.values.map(_.toString.toLowerCase).toSeq
 
   def toBaseRelation(table: FileStoreTable, _sqlContext: SQLContext): 
BaseRelation = {
     new BaseRelation {
diff --git 
a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
 
b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
index 91f7d1035f..8279a6de31 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
+++ 
b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.shim
 
 import org.apache.paimon.CoreOptions
 import org.apache.paimon.spark.SparkCatalog
+import org.apache.paimon.spark.catalog.FormatTableCatalog
 
 import org.apache.spark.sql.{SparkSession, Strategy}
 import org.apache.spark.sql.catalyst.analysis.ResolvedIdentifier
@@ -54,6 +55,20 @@ case class PaimonCreateTableAsSelectStrategy(spark: 
SparkSession)
             case (key, _) => coreOptionKeys.contains(key)
           }
           val newTableSpec = tableSpec.copy(properties = tableSpec.properties 
++ coreOptions)
+
+          val isPartitionedFormatTable = {
+            catalog match {
+              case catalog: FormatTableCatalog =>
+                catalog.isFormatTable(newTableSpec.provider.orNull) && 
parts.nonEmpty
+              case _ => false
+            }
+          }
+
+          if (isPartitionedFormatTable) {
+            throw new UnsupportedOperationException(
+              "Using CTAS with partitioned format table is not supported yet.")
+          }
+
           CreateTableAsSelectExec(
             catalog.asTableCatalog,
             ident,
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala
 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala
index 8ce0e178eb..8dcb1294ed 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala
+++ 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala
@@ -101,39 +101,6 @@ abstract class DDLTestBase extends PaimonSparkTestBase {
     }
   }
 
-  test("Paimon DDL: Create Table As Select") {
-    withTable("source", "t1", "t2") {
-      Seq((1L, "x1", "2023"), (2L, "x2", "2023"))
-        .toDF("a", "b", "pt")
-        .createOrReplaceTempView("source")
-
-      spark.sql("""
-                  |CREATE TABLE t1 AS SELECT * FROM source
-                  |""".stripMargin)
-      val t1 = loadTable("t1")
-      Assertions.assertTrue(t1.primaryKeys().isEmpty)
-      Assertions.assertTrue(t1.partitionKeys().isEmpty)
-
-      spark.sql(
-        """
-          |CREATE TABLE t2
-          |PARTITIONED BY (pt)
-          |TBLPROPERTIES ('bucket' = '5', 'primary-key' = 'a,pt', 
'target-file-size' = '128MB')
-          |AS SELECT * FROM source
-          |""".stripMargin)
-      val t2 = loadTable("t2")
-      Assertions.assertEquals(2, t2.primaryKeys().size())
-      Assertions.assertTrue(t2.primaryKeys().contains("a"))
-      Assertions.assertTrue(t2.primaryKeys().contains("pt"))
-      Assertions.assertEquals(1, t2.partitionKeys().size())
-      Assertions.assertEquals("pt", t2.partitionKeys().get(0))
-
-      // check all the core options
-      Assertions.assertEquals("5", t2.options().get("bucket"))
-      Assertions.assertEquals("128MB", t2.options().get("target-file-size"))
-    }
-  }
-
   test("Paimon DDL: create database with location with filesystem catalog") {
     withTempDir {
       dBLocation =>
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala
 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala
index 87b33e59b9..36315f0228 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala
+++ 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala
@@ -27,6 +27,8 @@ import org.junit.jupiter.api.Assertions
 
 abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase {
 
+  import testImplicits._
+
   test("Paimon DDL with hive catalog: create database with location and 
comment") {
     Seq(sparkCatalogName, paimonHiveCatalogName).foreach {
       catalogName =>
@@ -682,6 +684,49 @@ abstract class DDLWithHiveCatalogTestBase extends 
PaimonHiveTestBase {
     }
   }
 
+  test("Paimon DDL with hive catalog: Create Table As Select") {
+    Seq("paimon", sparkCatalogName, paimonHiveCatalogName).foreach {
+      catalogName =>
+        spark.sql(s"USE $catalogName")
+        withDatabase("paimon_db") {
+          spark.sql(s"CREATE DATABASE paimon_db")
+          spark.sql(s"USE paimon_db")
+
+          withTable("source", "t1", "t2") {
+            Seq((1L, "x1", "2023"), (2L, "x2", "2023"))
+              .toDF("a", "b", "pt")
+              .createOrReplaceTempView("source")
+
+            spark.sql("""
+                        |CREATE TABLE t1 USING paimon AS SELECT * FROM source
+                        |""".stripMargin)
+            val t1 = loadTable("paimon_db", "t1")
+            Assertions.assertTrue(t1.primaryKeys().isEmpty)
+            Assertions.assertTrue(t1.partitionKeys().isEmpty)
+
+            spark.sql(
+              """
+                |CREATE TABLE t2
+                |USING paimon
+                |PARTITIONED BY (pt)
+                |TBLPROPERTIES ('bucket' = '5', 'primary-key' = 'a,pt', 
'target-file-size' = '128MB')
+                |AS SELECT * FROM source
+                |""".stripMargin)
+            val t2 = loadTable("paimon_db", "t2")
+            Assertions.assertEquals(2, t2.primaryKeys().size())
+            Assertions.assertTrue(t2.primaryKeys().contains("a"))
+            Assertions.assertTrue(t2.primaryKeys().contains("pt"))
+            Assertions.assertEquals(1, t2.partitionKeys().size())
+            Assertions.assertEquals("pt", t2.partitionKeys().get(0))
+
+            // check all the core options
+            Assertions.assertEquals("5", t2.options().get("bucket"))
+            Assertions.assertEquals("128MB", 
t2.options().get("target-file-size"))
+          }
+        }
+    }
+  }
+
   def getDatabaseProp(dbName: String, propertyName: String): String = {
     spark
       .sql(s"DESC DATABASE EXTENDED $dbName")
diff --git 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/FormatTableTestBase.scala
 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/FormatTableTestBase.scala
index 6243d6b181..56c14aa64b 100644
--- 
a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/FormatTableTestBase.scala
+++ 
b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/FormatTableTestBase.scala
@@ -22,17 +22,15 @@ import org.apache.paimon.catalog.Identifier
 import org.apache.paimon.fs.Path
 import org.apache.paimon.spark.PaimonHiveTestBase
 import org.apache.paimon.table.FormatTable
-import org.apache.paimon.utils.{CompressUtils, FileIOUtils, FileUtils}
+import org.apache.paimon.utils.CompressUtils
 
 import org.apache.spark.sql.Row
 
-import java.io.{File, FileInputStream, FileOutputStream}
-import java.util.zip.GZIPOutputStream
-
 abstract class FormatTableTestBase extends PaimonHiveTestBase {
 
   override protected def beforeEach(): Unit = {
     sql(s"USE $paimonHiveCatalogName")
+    sql(s"USE $hiveDbName")
   }
 
   test("Format table: write partitioned table") {
@@ -47,7 +45,7 @@ abstract class FormatTableTestBase extends PaimonHiveTestBase 
{
 
         // check partition in file system
         val table =
-          paimonCatalog.getTable(Identifier.create("default", 
"t")).asInstanceOf[FormatTable]
+          paimonCatalog.getTable(Identifier.create(hiveDbName, 
"t")).asInstanceOf[FormatTable]
         val dirs = table.fileIO().listStatus(new 
Path(table.location())).map(_.getPath.getName)
         assert(dirs.count(_.startsWith("p1=")) == 1)
 
@@ -60,6 +58,22 @@ abstract class FormatTableTestBase extends 
PaimonHiveTestBase {
     }
   }
 
+  test("Format table: CTAS with partitioned table") {
+    withTable("t1", "t2") {
+      sql("CREATE TABLE t1 (id INT, p1 INT, p2 INT) USING csv PARTITIONED BY 
(p1, p2)")
+      sql("INSERT INTO t1 VALUES (1, 2, 3)")
+
+      assertThrows[UnsupportedOperationException] {
+        sql("""
+              |CREATE TABLE t2
+              |USING csv
+              |PARTITIONED BY (p1, p2)
+              |AS SELECT * FROM t1
+              |""".stripMargin)
+      }
+    }
+  }
+
   test("Format table: read compressed files") {
     for (format <- Seq("csv", "json")) {
       withTable("compress_t") {
@@ -67,7 +81,7 @@ abstract class FormatTableTestBase extends PaimonHiveTestBase 
{
         sql("INSERT INTO compress_t VALUES (1, 2, 3)")
         val table =
           paimonCatalog
-            .getTable(Identifier.create("default", "compress_t"))
+            .getTable(Identifier.create(hiveDbName, "compress_t"))
             .asInstanceOf[FormatTable]
         val fileIO = table.fileIO()
         val file = fileIO

Reply via email to