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

commit a098b72eafada848e8578f3a54c7fbb69b567939
Author: JingsongLi <[email protected]>
AuthorDate: Mon Apr 21 12:12:24 2025 +0800

    [test] Refactor SparkCatalogWithHiveTest and add partition tests
---
 .../java/org/apache/paimon/hive/HiveCatalog.java   |   6 +-
 .../paimon/spark/SparkCatalogWithHiveTest.java     | 109 ++++++++++-----------
 2 files changed, 56 insertions(+), 59 deletions(-)

diff --git 
a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
 
b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
index 164b27db56..07a22acea5 100644
--- 
a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
+++ 
b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
@@ -1467,7 +1467,11 @@ public class HiveCatalog extends AbstractCatalog {
         sd.setSerdeInfo(serDeInfo);
 
         CoreOptions options = new CoreOptions(schema.options());
-        if (options.partitionedTableInMetastore() && 
!schema.partitionKeys().isEmpty()) {
+        boolean partitioned = !schema.partitionKeys().isEmpty();
+        if (provider == null && !options.partitionedTableInMetastore()) {
+            partitioned = false;
+        }
+        if (partitioned) {
             Map<String, DataField> fieldMap =
                     schema.fields().stream()
                             .collect(Collectors.toMap(DataField::name, 
Function.identity()));
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 bb05e5224d..d6908259cc 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
@@ -55,30 +55,30 @@ public class SparkCatalogWithHiveTest {
         testHiveMetastore.stop();
     }
 
-    @Test
-    public void testCreateFormatTable(@TempDir java.nio.file.Path tempDir) {
+    @TempDir java.nio.file.Path tempDir;
+
+    private SparkSession.Builder createSessionBuilder() {
         Path warehousePath = new Path("file:" + tempDir.toString());
-        SparkSession spark =
-                SparkSession.builder()
-                        .config("spark.sql.warehouse.dir", 
warehousePath.toString())
-                        // with hive metastore
-                        .config("spark.sql.catalogImplementation", "hive")
-                        .config("hive.metastore.uris", "thrift://localhost:" + 
PORT)
-                        .config("spark.sql.catalog.spark_catalog", 
SparkCatalog.class.getName())
-                        .config("spark.sql.catalog.spark_catalog.metastore", 
"hive")
-                        .config(
-                                
"spark.sql.catalog.spark_catalog.hive.metastore.uris",
-                                "thrift://localhost:" + PORT)
-                        
.config("spark.sql.catalog.spark_catalog.format-table.enabled", "true")
-                        .config(
-                                "spark.sql.catalog.spark_catalog.warehouse",
-                                warehousePath.toString())
-                        .config(
-                                "spark.sql.extensions",
-                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
-                        .master("local[2]")
-                        .getOrCreate();
+        return SparkSession.builder()
+                .config("spark.sql.warehouse.dir", warehousePath.toString())
+                // with hive metastore
+                .config("spark.sql.catalogImplementation", "hive")
+                .config("hive.metastore.uris", "thrift://localhost:" + PORT)
+                .config("spark.sql.catalog.spark_catalog", 
SparkCatalog.class.getName())
+                .config("spark.sql.catalog.spark_catalog.metastore", "hive")
+                .config(
+                        "spark.sql.catalog.spark_catalog.hive.metastore.uris",
+                        "thrift://localhost:" + PORT)
+                .config("spark.sql.catalog.spark_catalog.warehouse", 
warehousePath.toString())
+                .config(
+                        "spark.sql.extensions",
+                        
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
+                .master("local[2]");
+    }
 
+    @Test
+    public void testCreateFormatTable() {
+        SparkSession spark = createSessionBuilder().getOrCreate();
         spark.sql("CREATE DATABASE IF NOT EXISTS my_db1");
         spark.sql("USE spark_catalog.my_db1");
 
@@ -131,23 +131,14 @@ public class SparkCatalogWithHiveTest {
     }
 
     @Test
-    public void testSpecifyHiveConfDir(@TempDir java.nio.file.Path tempDir) {
-        Path warehousePath = new Path("file:" + tempDir.toString());
+    public void testSpecifyHiveConfDirInGenericCatalog() {
         SparkSession spark =
-                SparkSession.builder()
+                createSessionBuilder()
                         
.config("spark.sql.catalog.spark_catalog.hive-conf-dir", "nonExistentPath")
-                        .config("spark.sql.warehouse.dir", 
warehousePath.toString())
-                        // with hive metastore
-                        .config("spark.sql.catalogImplementation", "hive")
                         .config(
                                 "spark.sql.catalog.spark_catalog",
                                 SparkGenericCatalog.class.getName())
-                        .config(
-                                "spark.sql.extensions",
-                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
-                        .master("local[2]")
                         .getOrCreate();
-
         assertThatThrownBy(() -> spark.sql("CREATE DATABASE my_db"))
                 .rootCause()
                 .isInstanceOf(FileNotFoundException.class)
@@ -157,36 +148,17 @@ public class SparkCatalogWithHiveTest {
     }
 
     @Test
-    public void testCreateExternalTable(@TempDir java.nio.file.Path tempDir) {
-        Path warehousePath = new Path("file:" + tempDir.toString());
-        SparkSession spark =
-                SparkSession.builder()
-                        .config("spark.sql.warehouse.dir", 
warehousePath.toString())
-                        // with hive metastore
-                        .config("spark.sql.catalogImplementation", "hive")
-                        .config("hive.metastore.uris", "thrift://localhost:" + 
PORT)
-                        .config("spark.sql.catalog.spark_catalog", 
SparkCatalog.class.getName())
-                        .config("spark.sql.catalog.spark_catalog.metastore", 
"hive")
-                        .config(
-                                
"spark.sql.catalog.spark_catalog.hive.metastore.uris",
-                                "thrift://localhost:" + PORT)
-                        .config(
-                                "spark.sql.catalog.spark_catalog.warehouse",
-                                warehousePath.toString())
-                        .config(
-                                "spark.sql.extensions",
-                                
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
-                        .master("local[2]")
-                        .getOrCreate();
-
+    public void testCreateExternalTable() {
+        SparkSession spark = createSessionBuilder().getOrCreate();
+        String warehousePath = 
spark.sparkContext().conf().get("spark.sql.warehouse.dir");
         spark.sql("CREATE DATABASE IF NOT EXISTS test_db");
         spark.sql("USE spark_catalog.test_db");
 
         // create hive external table
-        spark.sql("CREATE EXTERNAL TABLE t1 (a INT, bb INT, c STRING)");
+        spark.sql("CREATE EXTERNAL TABLE external_table (a INT, bb INT, c 
STRING)");
 
         // drop hive external table
-        spark.sql("DROP TABLE t1");
+        spark.sql("DROP TABLE external_table");
 
         // file system table exists
         assertThatCode(
@@ -195,9 +167,30 @@ public class SparkCatalogWithHiveTest {
                                         LocalFileIO.create(),
                                         new Path(
                                                 warehousePath,
-                                                String.format("%s.db/%s", 
"test_db", "t1"))))
+                                                String.format(
+                                                        "%s.db/%s", "test_db", 
"external_table"))))
                 .doesNotThrowAnyException();
 
         spark.close();
     }
+
+    @Test
+    public void testPartitionedFormatTable() {
+        SparkSession spark = createSessionBuilder().getOrCreate();
+        spark.sql("CREATE DATABASE IF NOT EXISTS test_db");
+        spark.sql("USE spark_catalog.test_db");
+
+        spark.sql(
+                "CREATE TABLE part_format_table (a INT, b INT, dt STRING) 
USING CSV PARTITIONED BY (dt)");
+        spark.sql("INSERT INTO TABLE part_format_table VALUES (1, 1, '1'), (2, 
2, '2')");
+        assertThat(
+                        spark.sql("SELECT * FROM 
part_format_table").collectAsList().stream()
+                                .map(Row::toString)
+                                .collect(Collectors.toList()))
+                .containsExactlyInAnyOrder("[1,1,1]", "[2,2,2]");
+
+        // TODO assert partition paths, it should be true partitioned table
+
+        spark.close();
+    }
 }

Reply via email to