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

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


The following commit(s) were added to refs/heads/main by this push:
     new a80309f5d6 [GLUTEN-12181][CH] Fix issues for the Spark 3.5 + Delta 
Delete Vector with the CH backend (#12182)
a80309f5d6 is described below

commit a80309f5d6afeeeb142f03c8a0b94a0ef14732d4
Author: Zhichao Zhang <[email protected]>
AuthorDate: Fri May 29 14:23:38 2026 +0800

    [GLUTEN-12181][CH] Fix issues for the Spark 3.5 + Delta Delete Vector with 
the CH backend (#12182)
    
    Fix issues for the Spark 3.5 + Delta Delete Vector with the CH backend: 
enable the suite GlutenDeltaParquetDeletionVectorSuite
---
 .../GlutenDeltaMergeTreeDeletionVectorSuite.scala  | 293 ---------------------
 .../GlutenDeltaParquetDeletionVectorSuite.scala    |  39 ++-
 .../GlutenClickHouseDeltaParquetWriteSuite.scala   | 106 +++++++-
 .../GlutenClickHouseNativeWriteTableSuite.scala    |   4 +-
 .../Functions/LocalDigitsToAsciiDigitForDate.cpp   |   2 +-
 .../Parser/RelParsers/ReadRelParser.cpp            |   1 -
 .../Storages/SubstraitSource/Delta/DeltaWriter.cpp |   4 +-
 7 files changed, 144 insertions(+), 305 deletions(-)

diff --git 
a/backends-clickhouse/src-delta33/test/scala/org/apache/spark/gluten/delta/GlutenDeltaMergeTreeDeletionVectorSuite.scala
 
b/backends-clickhouse/src-delta33/test/scala/org/apache/spark/gluten/delta/GlutenDeltaMergeTreeDeletionVectorSuite.scala
deleted file mode 100644
index 7695a16a7c..0000000000
--- 
a/backends-clickhouse/src-delta33/test/scala/org/apache/spark/gluten/delta/GlutenDeltaMergeTreeDeletionVectorSuite.scala
+++ /dev/null
@@ -1,293 +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.spark.gluten.delta
-
-import org.apache.gluten.backendsapi.clickhouse.CHConfig
-import org.apache.gluten.config.GlutenConfig
-import org.apache.gluten.execution.{CreateMergeTreeSuite, 
FileSourceScanExecTransformer}
-
-import org.apache.spark.SparkConf
-import org.apache.spark.sql.delta.files.TahoeFileIndex
-import org.apache.spark.sql.delta.stats.PreparedDeltaFileIndex
-import 
org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata.AddMergeTreeParts
-
-// Some sqls' line length exceeds 100
-// scalastyle:off line.size.limit
-
-class GlutenDeltaMergeTreeDeletionVectorSuite extends CreateMergeTreeSuite {
-
-  /** Run Gluten + ClickHouse Backend with SortShuffleManager */
-  override protected def sparkConf: SparkConf = {
-    super.sparkConf
-      .set("spark.shuffle.manager", 
"org.apache.spark.shuffle.sort.ColumnarShuffleManager")
-      .set("spark.io.compression.codec", "LZ4")
-      .set("spark.sql.shuffle.partitions", "5")
-      .set("spark.sql.autoBroadcastJoinThreshold", "10MB")
-      .set("spark.sql.adaptive.enabled", "true")
-      .set("spark.sql.storeAssignmentPolicy", "legacy")
-      .set("spark.databricks.delta.retentionDurationCheck.enabled", "false")
-      .set(GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")
-      .set(CHConfig.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, spark35.toString)
-  }
-
-  test("Gluten-9334: column `_tmp_metadata_row_index` and `file_path` not 
found") {
-    val tableName = "delta_metadata_column"
-    withTable(tableName) {
-      withTempDir {
-        dirName =>
-          val s = createTableBuilder(tableName, "clickhouse", 
s"$dirName/$tableName")
-            .withProps(Map("delta.enableDeletionVectors" -> "'true'"))
-            .withTableKey("lineitem")
-            .build()
-          spark.sql(s)
-
-          spark.sql(s"insert into table $tableName select * from lineitem ")
-
-          val df = sql(s"""
-                          | select
-                          |   _metadata.file_path,
-                          |   _metadata.row_index
-                          | from $tableName
-                          | limit 1
-                          |""".stripMargin)
-
-          checkFallbackOperators(df, 0)
-      }
-    }
-  }
-
-  ignore("Gluten-9606: Support CH MergeTree + Delta DeletionVector reading") {
-    val tableName = "mergetree_delta_dv"
-    withTable(tableName) {
-      withTempDir {
-        dirName =>
-          val s = createTableBuilder(tableName, "clickhouse", 
s"$dirName/$tableName")
-            .withProps(Map("delta.enableDeletionVectors" -> "'true'"))
-            .withTableKey("lineitem")
-            .build()
-          spark.sql(s)
-
-          spark.sql(s"""
-                       |insert into table $tableName
-                       |select /*+ REPARTITION(6) */ * from lineitem
-                       |""".stripMargin)
-
-          spark.sql(s"""
-                       | delete from $tableName
-                       | where l_orderkey = 3
-                       |""".stripMargin)
-
-          val df = spark.sql(s"""
-                                | select sum(l_linenumber) from $tableName
-                                |""".stripMargin)
-          val result = df.collect()
-          assert(
-            result.apply(0).get(0) === 1802425
-          )
-          checkFallbackOperators(df, 0)
-
-          spark.sql(s"""
-                       | delete from $tableName
-                       | where mod(l_orderkey, 3) = 2
-                       |""".stripMargin)
-
-          val df1 = spark.sql(s"""
-                                 | select sum(l_linenumber) from $tableName
-                                 |""".stripMargin)
-          assert(
-            df1.collect().apply(0).get(0) === 1200650
-          )
-          checkFallbackOperators(df1, 0)
-
-          spark.sql(s"optimize $tableName")
-
-          val df2 = spark.sql(s"""
-                                 | select sum(l_linenumber) from $tableName
-                                 |""".stripMargin)
-          val result2 = df2.collect()
-          assert(result2.apply(0).get(0) === 1200650)
-
-          val scanExec = collect(df2.queryExecution.executedPlan) {
-            case f: FileSourceScanExecTransformer => f
-          }
-          assertResult(1)(scanExec.size)
-          val mergetreeScan = scanExec.head
-          
assert(mergetreeScan.nodeName.startsWith("FileSourceScanExecTransformer 
mergetree"))
-          val fileIndex = 
mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex]
-          val addFiles =
-            fileIndex.matchingFiles(Nil, Nil).map(f => 
f.asInstanceOf[AddMergeTreeParts])
-          assertResult(1)(addFiles.size)
-      }
-    }
-  }
-
-  ignore("Gluten-9606: Support CH MergeTree + Delta DeletionVector reading -- 
partition") {
-    val tableName = "mergetree_delta_dv_partition"
-    spark.sql(s"""
-                 |DROP TABLE IF EXISTS $tableName;
-                 |""".stripMargin)
-    spark.sql(s"""
-                 |CREATE TABLE IF NOT EXISTS $tableName
-                 |(${table2columns.get("lineitem").get(true)})
-                 |USING clickhouse
-                 |PARTITIONED BY (l_returnflag)
-                 |TBLPROPERTIES (delta.enableDeletionVectors='true')
-                 |LOCATION '$dataHome/$tableName'
-                 |""".stripMargin)
-
-    spark.sql(s"""
-                 | insert into table $tableName
-                 | select /*+ REPARTITION(6) */ * from lineitem
-                 |""".stripMargin)
-
-    spark.sql(s"""
-                 | delete from $tableName
-                 | where mod(l_orderkey, 3) = 1
-                 |""".stripMargin)
-
-    var df = spark.sql(s"""
-                          | select sum(l_linenumber) from $tableName
-                          | where mod(l_orderkey, 3) = 1
-                          |""".stripMargin)
-    var result = df.collect()
-    assert(
-      result.apply(0).isNullAt(0)
-    )
-    checkFallbackOperators(df, 0)
-    df = spark.sql(s"""
-                      | select sum(l_linenumber) from $tableName
-                      | where mod(l_orderkey, 3) = 2
-                      |""".stripMargin)
-    result = df.collect()
-    assert(
-      result.apply(0).get(0) === 601775
-    )
-    checkFallbackOperators(df, 0)
-    df = spark.sql(s"""
-                      | select sum(l_linenumber) from $tableName
-                      |""".stripMargin)
-    result = df.collect()
-    assert(
-      result.apply(0).get(0) === 1201486
-    )
-    checkFallbackOperators(df, 0)
-  }
-
-  // TODO fix reorg purge command in delta 3.3.1
-  ignore("Gluten-9697: Add 'reorg' command ut for the mergetree + delta dv") {
-    val tableName = "mergetree_delta_dv_reorg"
-    withTable(tableName) {
-      withTempDir {
-        dirName =>
-          val s = createTableBuilder(tableName, "clickhouse", 
s"$dirName/$tableName")
-            .withProps(Map("delta.enableDeletionVectors" -> "'true'"))
-            .withTableKey("lineitem")
-            .build()
-          spark.sql(s)
-
-          spark.sql(s"""
-                       |insert into table $tableName
-                       |select /*+ REPARTITION(6) */ * from lineitem
-                       |""".stripMargin)
-
-          spark.sql(s"""
-                       |delete from $tableName
-                       |where mod(l_orderkey, 3) = 2
-                       |""".stripMargin)
-
-          var df = spark.sql(s"""
-                                | select sum(l_linenumber) from $tableName
-                                |""".stripMargin)
-          var result = df.collect()
-          assert(
-            result(0).get(0) === 1200671
-          )
-          checkFallbackOperators(df, 0)
-
-          spark.sql(s"""
-                       | REORG TABLE $tableName APPLY (PURGE)
-                       |""".stripMargin)
-          df = spark.sql(s"""
-                            | select sum(l_linenumber) from $tableName
-                            |""".stripMargin)
-          result = df.collect()
-          assert(
-            result(0).get(0) === 1200671
-          )
-          val scanExec = collect(df.queryExecution.executedPlan) {
-            case f: FileSourceScanExecTransformer => f
-          }
-          val parquetScan = scanExec.head
-          val fileIndex = 
parquetScan.relation.location.asInstanceOf[PreparedDeltaFileIndex]
-          val addFiles = fileIndex.preparedScan.files
-          assert(addFiles.size === 1)
-          assert(addFiles(0).deletionVector === null)
-      }
-    }
-  }
-
-  // TODO fix reorg purge command in delta 3.3.1
-  ignore("Gluten-9697: Add 'reorg' command ut for the mergetree + delta dv + 
partition") {
-    val tableName = "mergetree_delta_dv_reorg_partition"
-    spark.sql(s"""
-                 |DROP TABLE IF EXISTS $tableName;
-                 |""".stripMargin)
-    spark.sql(s"""
-                 |CREATE TABLE IF NOT EXISTS $tableName
-                 |(${table2columns.get("lineitem").get(true)})
-                 |USING clickhouse
-                 |PARTITIONED BY (l_returnflag)
-                 |TBLPROPERTIES (delta.enableDeletionVectors='true')
-                 |LOCATION '$dataHome/$tableName'
-                 |""".stripMargin)
-    spark.sql(s"""
-                 |insert into table $tableName
-                 | select /*+ REPARTITION(6) */ * from lineitem
-                 |""".stripMargin)
-    spark.sql(s"""
-                 |delete from $tableName
-                 | where mod(l_orderkey, 3) = 1
-                 |""".stripMargin)
-    var df = spark.sql(s"""
-                          |select sum(l_linenumber) from $tableName
-                          |""".stripMargin)
-    var result = df.collect()
-    assert(
-      result(0).get(0) === 1201486
-    )
-    checkFallbackOperators(df, 0)
-    spark.sql(s"""
-                 |REORG TABLE $tableName APPLY (PURGE)
-                 |""".stripMargin)
-    df = spark.sql(s"""
-                      |select sum(l_linenumber) from $tableName
-                      |""".stripMargin)
-    result = df.collect()
-    assert(
-      result(0).get(0) === 1201486
-    )
-    val scanExec = collect(df.queryExecution.executedPlan) {
-      case f: FileSourceScanExecTransformer => f
-    }
-    val parquetScan = scanExec.head
-    val fileIndex = 
parquetScan.relation.location.asInstanceOf[PreparedDeltaFileIndex]
-    val addFiles = fileIndex.preparedScan.files
-    assert(addFiles.size === 3)
-    assert(addFiles.forall(_.deletionVector === null))
-  }
-}
-// scalastyle:off line.size.limit
diff --git 
a/backends-clickhouse/src-delta33/test/scala/org/apache/spark/gluten/delta/GlutenDeltaParquetDeletionVectorSuite.scala
 
b/backends-clickhouse/src-delta33/test/scala/org/apache/spark/gluten/delta/GlutenDeltaParquetDeletionVectorSuite.scala
index c1be2af625..5f94940c6d 100644
--- 
a/backends-clickhouse/src-delta33/test/scala/org/apache/spark/gluten/delta/GlutenDeltaParquetDeletionVectorSuite.scala
+++ 
b/backends-clickhouse/src-delta33/test/scala/org/apache/spark/gluten/delta/GlutenDeltaParquetDeletionVectorSuite.scala
@@ -67,7 +67,34 @@ class GlutenDeltaParquetDeletionVectorSuite extends 
ParquetSuite {
        | l_shipmode      string,
        | l_comment       string""".stripMargin
 
-  ignore("test parquet table delete with the delta DV") {
+  test("Gluten-9334: column `_tmp_metadata_row_index` and `file_path` not 
found") {
+    val tableName = "delta_metadata_column"
+    withTable(tableName) {
+      withTempDir {
+        dirName =>
+          val s = createTableBuilder(tableName, "delta", 
s"$dirName/$tableName")
+            .withProps(Map("delta.enableDeletionVectors" -> "'true'"))
+            .withTableKey("lineitem")
+            .build()
+          spark.sql(s)
+
+          spark.sql(s"insert into table $tableName select * from lineitem ")
+
+          val df = sql(
+            s"""
+               | select
+               |   _metadata.file_path,
+               |   _metadata.row_index
+               | from $tableName
+
+               |""".stripMargin)
+
+          checkFallbackOperators(df, 0)
+      }
+    }
+  }
+
+  test("test parquet table delete with the delta DV") {
     spark.sql(s"""
                  |DROP TABLE IF EXISTS lineitem_delta_parquet_delete_dv;
                  |""".stripMargin)
@@ -117,7 +144,7 @@ class GlutenDeltaParquetDeletionVectorSuite extends 
ParquetSuite {
     )
   }
 
-  ignore("test parquet table delete + update with the delta DV") {
+  test("test parquet table delete + update with the delta DV") {
     spark.sql(s"""
                  |DROP TABLE IF EXISTS lineitem_delta_parquet_update_dv;
                  |""".stripMargin)
@@ -193,7 +220,7 @@ class GlutenDeltaParquetDeletionVectorSuite extends 
ParquetSuite {
     }
   }
 
-  ignore("test delta DV write") {
+  test("test delta DV write") {
     val table_name = "dv_write_test"
     withTable(table_name) {
       spark.sql(s"""
@@ -297,7 +324,7 @@ class GlutenDeltaParquetDeletionVectorSuite extends 
ParquetSuite {
   }
 
   for (targetDVFileSize <- Seq(2, 200, 2000000)) {
-    ignore(
+    test(
       s"DELETE with DVs - packing multiple DVs into one file: target max DV 
file " +
         s"size=$targetDVFileSize") {
       withSQLConf(
@@ -345,7 +372,7 @@ class GlutenDeltaParquetDeletionVectorSuite extends 
ParquetSuite {
     }
   }
 
-  ignore("test parquet partition table delete with the delta DV") {
+  test("test parquet partition table delete with the delta DV") {
     withSQLConf(("spark.sql.sources.partitionOverwriteMode", "dynamic")) {
       spark.sql(s"""
                    |DROP TABLE IF EXISTS 
lineitem_delta_partition_parquet_delete_dv;
@@ -385,7 +412,7 @@ class GlutenDeltaParquetDeletionVectorSuite extends 
ParquetSuite {
     }
   }
 
-  ignore("test parquet table upsert with the delta DV") {
+  test("test parquet table upsert with the delta DV") {
     spark.sql(s"""
                  |DROP TABLE IF EXISTS lineitem_delta_parquet_upsert_dv;
                  |""".stripMargin)
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala
index d0bd2b947c..e85a4d52ab 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala
@@ -23,6 +23,7 @@ import org.apache.spark.SparkConf
 import org.apache.spark.gluten.delta.DeltaStatsUtils
 import org.apache.spark.sql.SaveMode
 import org.apache.spark.sql.delta.files.TahoeFileIndex
+import org.apache.spark.sql.delta.stats.PreparedDeltaFileIndex
 
 import io.delta.tables.DeltaTable
 
@@ -542,7 +543,7 @@ class GlutenClickHouseDeltaParquetWriteSuite extends 
ParquetTPCHSuite {
                  |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_ctas2
                  |USING delta
                  |PARTITIONED BY (l_returnflag)
-                 |LOCATION '$dataHome/lineitem_mergetree_ctas2'
+                 |LOCATION '$dataHome/lineitem_delta_parquet_ctas2'
                  | as select * from lineitem
                  |""".stripMargin)
     checkQuery(q1("lineitem_delta_parquet_ctas2"))
@@ -1082,5 +1083,108 @@ class GlutenClickHouseDeltaParquetWriteSuite extends 
ParquetTPCHSuite {
     val ret = spark.sql(s"select count(*) from 
clickhouse.`$dataPath`").collect()
     assert(ret.apply(0).get(0) === 600572)
   }
+
+  // TODO: after rebase-25.12, support 'reorg' command for delta dv + partition
+  ignore("Gluten-9697: Add 'reorg' command ut for the mergetree + delta dv") {
+    val tableName = "mergetree_delta_dv_reorg"
+    withTable(tableName) {
+      withTempDir {
+        dirName =>
+          val s = createTableBuilder(tableName, "delta", 
s"$dirName/$tableName")
+            .withProps(Map("delta.enableDeletionVectors" -> "'true'"))
+            .withTableKey("lineitem")
+            .build()
+          spark.sql(s)
+
+          spark.sql(s"""
+                       |insert into table $tableName
+                       |select /*+ REPARTITION(6) */ * from lineitem
+                       |""".stripMargin)
+
+          spark.sql(s"""
+                       |delete from $tableName
+                       |where mod(l_orderkey, 3) = 2
+                       |""".stripMargin)
+
+          var df = spark.sql(s"""
+                                | select sum(l_linenumber) from $tableName
+                                |""".stripMargin)
+          var result = df.collect()
+          assert(
+            result(0).get(0) === 1200671
+          )
+          checkFallbackOperators(df, 0)
+
+          spark.sql(s"""
+                       | REORG TABLE $tableName APPLY (PURGE)
+                       |""".stripMargin)
+          df = spark.sql(s"""
+                            | select sum(l_linenumber) from $tableName
+                            |""".stripMargin)
+          result = df.collect()
+          assert(
+            result(0).get(0) === 1200671
+          )
+          val scanExec = collect(df.queryExecution.executedPlan) {
+            case f: FileSourceScanExecTransformer => f
+          }
+          val parquetScan = scanExec.head
+          val fileIndex = 
parquetScan.relation.location.asInstanceOf[PreparedDeltaFileIndex]
+          val addFiles = fileIndex.preparedScan.files
+          assert(addFiles.size === 1)
+          assert(addFiles(0).deletionVector === null)
+      }
+    }
+  }
+
+  // TODO: after rebase-25.12, fix reorg purge command in delta 3.3.1
+  ignore("Gluten-9697: Add 'reorg' command ut for delta dv + partition") {
+    val tableName = "mergetree_delta_dv_reorg_partition"
+    spark.sql(s"""
+                 |DROP TABLE IF EXISTS $tableName;
+                 |""".stripMargin)
+    spark.sql(s"""
+                 |CREATE TABLE IF NOT EXISTS $tableName
+                 |(${table2columns.get("lineitem").get(true)})
+                 |USING delta
+                 |PARTITIONED BY (l_returnflag)
+                 |TBLPROPERTIES (delta.enableDeletionVectors='true')
+                 |LOCATION '$dataHome/$tableName'
+                 |""".stripMargin)
+    spark.sql(s"""
+                 |insert into table $tableName
+                 | select /*+ REPARTITION(6) */ * from lineitem
+                 |""".stripMargin)
+    spark.sql(s"""
+                 |delete from $tableName
+                 | where mod(l_orderkey, 3) = 1
+                 |""".stripMargin)
+    var df = spark.sql(s"""
+                          |select sum(l_linenumber) from $tableName
+                          |""".stripMargin)
+    var result = df.collect()
+    assert(
+      result(0).get(0) === 1201486
+    )
+    checkFallbackOperators(df, 0)
+    spark.sql(s"""
+                 |REORG TABLE $tableName APPLY (PURGE)
+                 |""".stripMargin)
+    df = spark.sql(s"""
+                      |select sum(l_linenumber) from $tableName
+                      |""".stripMargin)
+    result = df.collect()
+    assert(
+      result(0).get(0) === 1201486
+    )
+    val scanExec = collect(df.queryExecution.executedPlan) {
+      case f: FileSourceScanExecTransformer => f
+    }
+    val parquetScan = scanExec.head
+    val fileIndex = 
parquetScan.relation.location.asInstanceOf[PreparedDeltaFileIndex]
+    val addFiles = fileIndex.preparedScan.files
+    assert(addFiles.size === 3)
+    assert(addFiles.forall(_.deletionVector === null))
+  }
 }
 // scalastyle:off line.size.limit
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala
index 05b306a4f1..289a58106c 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala
@@ -327,7 +327,7 @@ class GlutenClickHouseNativeWriteTableSuite
     }
   }
 
-  // TODO: after rebase-25.12, failed with spark35 (Memory limit exceeded), 
fix later
+  // TODO: after rebase-25.12, failed with spark35 + Orc (Memory limit 
exceeded), fix later
   testWithSpecifiedSparkVersion("test 2-col partitioned table", "3.3") {
     val fields: ListMap[String, String] = ListMap(
       ("string_field", "string"),
@@ -551,6 +551,7 @@ class GlutenClickHouseNativeWriteTableSuite
     }
   }
 
+  // TODO: after rebase-25.12, failed with spark35 + Orc (Memory limit 
exceeded), fix later
   testWithMaxSparkVersion("test 1-col partitioned + 2-col bucketed table", 
"3.3") {
     val fields: ListMap[String, String] = ListMap(
       ("string_field", "string"),
@@ -625,6 +626,7 @@ class GlutenClickHouseNativeWriteTableSuite
     }
   }
 
+  // TODO: after rebase-25.12, failed with spark35 + Orc (Memory limit 
exceeded), fix later
   testWithMaxSparkVersion("test decimal with rand()", "3.3") {
     nativeWrite {
       format =>
diff --git a/cpp-ch/local-engine/Functions/LocalDigitsToAsciiDigitForDate.cpp 
b/cpp-ch/local-engine/Functions/LocalDigitsToAsciiDigitForDate.cpp
index e260c52b74..4474a2eb35 100644
--- a/cpp-ch/local-engine/Functions/LocalDigitsToAsciiDigitForDate.cpp
+++ b/cpp-ch/local-engine/Functions/LocalDigitsToAsciiDigitForDate.cpp
@@ -169,7 +169,7 @@ public:
                 LOG_DEBUG(
                     getLogger("LocalDigitsToAsciiDigitForDateFunction"),
                     "Converted local digit string {} to ascii digit string: 
{}",
-                    String(col_str->getDataAt(row_index).data(), 
col_str->getDataAt(row_index).size()),
+                    col_str->getDataAt(row_index),
                     converted);
                 res_col->insertData(converted.c_str(), converted.size());
             }
diff --git a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp 
b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp
index 00b8062d77..9819dc2c01 100644
--- a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp
+++ b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp
@@ -230,7 +230,6 @@ QueryPlanStepPtr ReadRelParser::parseReadRelWithRange(const 
substrait::ReadRel &
 
     chassert(extension_table.has_detail());
     std::string str_range_info = toString(extension_table.detail());
-    // std::cout << "range_info:" << str_range_info << std::endl;
 
     rapidjson::Document document;
     document.Parse(str_range_info.c_str());
diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/Delta/DeltaWriter.cpp 
b/cpp-ch/local-engine/Storages/SubstraitSource/Delta/DeltaWriter.cpp
index 47c6de6188..b593966821 100644
--- a/cpp-ch/local-engine/Storages/SubstraitSource/Delta/DeltaWriter.cpp
+++ b/cpp-ch/local-engine/Storages/SubstraitSource/Delta/DeltaWriter.cpp
@@ -190,7 +190,7 @@ void DeltaWriter::writeDeletionVector(const DB::Block & 
block)
                 {
                     // use already existing deletion vector
                     auto dv_descriptor_field = 
existing_deletion_vector_descriptor.createDeletionVectorDescriptorField();
-                    file_path_column->insert(file_path.data());
+                    file_path_column->insert(file_path);
                     dv_descriptor_column->insert(dv_descriptor_field);
                     matched_row_count_col->insert(cardinality);
                     continue;
@@ -214,7 +214,7 @@ void DeltaWriter::writeDeletionVector(const DB::Block & 
block)
         DeletionVectorDescriptor descriptor(cardinality, bitmap_size, 
DeletionVectorDescriptor::UUID_DV_MARKER, offset, prefix + encoded);
         auto dv_descriptor_field = 
descriptor.createDeletionVectorDescriptorField();
 
-        file_path_column->insert(file_path.data());
+        file_path_column->insert(file_path);
         dv_descriptor_column->insert(dv_descriptor_field);
         matched_row_count_col->insert(cardinality);
 


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

Reply via email to