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

zhangzc pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
     new 5ad819800  [GLUTEN-6604][CH] Fix mergetree partition with whitespace 
error (#6605)
5ad819800 is described below

commit 5ad819800739766565266402c21e51a1667c0583
Author: Shuai li <[email protected]>
AuthorDate: Fri Jul 26 14:04:36 2024 +0800

     [GLUTEN-6604][CH] Fix mergetree partition with whitespace error (#6605)
    
    [CH] Fix mergetree partition with whitespace error
---
 .../utils/MergeTreePartsPartitionsUtil.scala       |  2 +-
 .../clickhouse/MergeTreeFileFormatDataWriter.scala |  3 ++-
 .../GlutenClickHouseMergeTreeWriteSuite.scala      | 28 ++++++++++++++++++++++
 3 files changed, 31 insertions(+), 2 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala
 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala
index 64e41778c..03199f7ff 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala
@@ -166,7 +166,7 @@ object MergeTreePartsPartitionsUtil extends Logging {
         partition =>
           partition.files.map(
             fs => {
-              val path = fs.getPath.toString
+              val path = fs.getPath.toUri.toString
 
               val ret = ClickhouseSnapshot.pathToAddMTPCache.getIfPresent(path)
               if (ret == null) {
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatDataWriter.scala
 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatDataWriter.scala
index 712afb378..4f522e218 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatDataWriter.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatDataWriter.scala
@@ -306,7 +306,8 @@ abstract class MergeTreeBaseDynamicPartitionDataWriter(
       releaseCurrentWriter()
     }
 
-    val partDir = partitionValues.map(getPartitionPath(_))
+    val partDir =
+      partitionValues.map(getPartitionPath(_)).map(str => new 
Path(str).toUri.toASCIIString)
     partDir.foreach(updatedPartitions.add)
 
     val bucketIdStr = bucketId.map(id => f"$id%05d").getOrElse("")
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala
index 2563d792b..77d7f37c0 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala
@@ -2022,5 +2022,33 @@ class GlutenClickHouseMergeTreeWriteSuite
          |""".stripMargin
     runTPCHQueryBySQL(6, sqlStr) { _ => }
   }
+
+  test("test mergetree with partition with whitespace") {
+    spark.sql(s"""
+                 |DROP TABLE IF EXISTS 
lineitem_mergetree_partition_with_whitespace;
+                 |""".stripMargin)
+
+    spark.sql(s"""
+                 |CREATE TABLE IF NOT EXISTS 
lineitem_mergetree_partition_with_whitespace
+                 |(
+                 | l_returnflag    string,
+                 | l_linestatus    string
+                 |)
+                 |USING clickhouse
+                 |PARTITIONED BY (l_returnflag)
+                 |LOCATION 
'$basePath/lineitem_mergetree_partition_with_whitespace'
+                 |""".stripMargin)
+
+    spark.sql(s"""
+                 | insert into table 
lineitem_mergetree_partition_with_whitespace
+                 | (l_returnflag, l_linestatus) values ('a A', 'abc')
+                 |""".stripMargin)
+
+    val sqlStr =
+      s"""
+         |SELECT * from lineitem_mergetree_partition_with_whitespace
+         |""".stripMargin
+    runSql(sqlStr) { _ => }
+  }
 }
 // scalastyle:off line.size.limit


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to