From e1bcf99f849eb47c1019d99a4f5293168e3ce994 Mon Sep 17 00:00:00 2001 From: Zhichao Zhang Date: Wed, 11 Mar 2026 09:39:55 +0800 Subject: [PATCH 1/3] [CH]Update Clickhouse Version (Branch_25.12_20260310) 1. Update Clickhouse version to branch 25.12(20260310) 2. [GLUTEN-11488][CH] Support aggregate function and align sparkArrayFold lambda types --- .../execution/GlutenClickHouseTPCHSuite.scala | 4 +- .../GlutenFunctionValidateSuite.scala | 53 ++++ .../execution/GlutenNothingValueCheck.scala | 8 +- ...utenClickHouseMergeTreeOptimizeSuite.scala | 36 +-- ...seTPCHColumnarShuffleParquetAQESuite.scala | 29 ++ ...utenClickHouseTPCHParquetBucketSuite.scala | 4 +- cpp-ch/clickhouse.version | 4 +- cpp-ch/local-engine/CMakeLists.txt | 3 +- cpp-ch/local-engine/Common/AggregateUtil.cpp | 6 +- .../local-engine/Common/ArrayJoinHelper.cpp | 4 +- cpp-ch/local-engine/Common/CHUtil.cpp | 8 +- cpp-ch/local-engine/Common/CHUtil.h | 2 + cpp-ch/local-engine/Common/DebugUtils.cpp | 5 +- cpp-ch/local-engine/Common/PlanUtil.cpp | 23 +- cpp-ch/local-engine/Common/PlanUtil.h | 2 +- cpp-ch/local-engine/Common/QueryContext.cpp | 13 +- .../CompactObjectStorageDiskTransaction.cpp | 28 +- .../CompactObjectStorageDiskTransaction.h | 20 +- .../Disks/ObjectStorages/GlutenDiskHDFS.h | 4 +- .../Disks/ObjectStorages/GlutenDiskS3.cpp | 2 +- .../Disks/ObjectStorages/GlutenDiskS3.h | 2 +- .../GlutenHDFSObjectStorage.cpp | 8 - .../ObjectStorages/GlutenHDFSObjectStorage.h | 6 +- .../MetadataStorageFromRocksDB.cpp | 39 ++- .../MetadataStorageFromRocksDB.h | 18 +- ...torageFromRocksDBTransactionOperations.cpp | 24 +- ...aStorageFromRocksDBTransactionOperations.h | 28 +- .../registerGlutenDiskObjectStorage.cpp | 9 +- .../Disks/registerGlutenDisks.cpp | 6 +- .../AggregateFunctionSparkArrayFold.cpp | 297 ++++++++++++++++++ .../Functions/FunctionGetDateData.h | 4 +- .../Functions/FunctionsBloomFilter.h | 6 +- .../LocalDigitsToAsciiDigitForDate.cpp | 26 +- .../Functions/SparkFunctionArrayDistinct.cpp | 6 +- .../Functions/SparkFunctionArrayJoin.cpp | 19 +- .../Functions/SparkFunctionConv.cpp | 3 +- .../Functions/SparkFunctionGetJsonObject.cpp | 20 +- .../Functions/SparkFunctionGetJsonObject.h | 10 +- .../Functions/SparkFunctionHashingExtended.h | 8 +- .../Functions/SparkFunctionMonthsBetween.cpp | 4 +- .../SparkFunctionReinterpretAsString.cpp | 3 +- .../Functions/SparkFunctionStrToMap.cpp | 6 +- .../Functions/SparkFunctionToDateTime.h | 4 +- .../Functions/SparkFunctionTrim.cpp | 16 +- .../local-engine/Functions/SparkParseURL.cpp | 3 +- .../Functions/SparkPartitionEscape.cpp | 3 +- .../IO/SplittableBzip2ReadBuffer.h | 1 - .../FillingDeltaInternalRowDeletedStep.cpp | 3 +- .../Operator/GraceAggregatingTransform.cpp | 4 +- .../Parser/AggregateFunctionParser.cpp | 5 +- .../Parser/CHColumnToSparkRow.cpp | 25 +- .../local-engine/Parser/CHColumnToSparkRow.h | 4 +- .../local-engine/Parser/ExpressionParser.cpp | 4 +- cpp-ch/local-engine/Parser/FunctionParser.cpp | 3 +- cpp-ch/local-engine/Parser/RelMetric.cpp | 2 +- .../Parser/RelParsers/AggregateRelParser.cpp | 3 +- .../Parser/RelParsers/CrossRelParser.cpp | 6 +- .../Parser/RelParsers/FilterRelParser.cpp | 2 +- .../Parser/RelParsers/GroupLimitRelParser.cpp | 11 +- .../Parser/RelParsers/JoinRelParser.cpp | 7 +- .../Parser/RelParsers/MergeTreeRelParser.cpp | 6 +- .../Parser/RelParsers/SortRelParser.cpp | 3 +- .../Parser/RelParsers/WindowRelParser.cpp | 4 +- .../Parser/RelParsers/WriteRelParser.cpp | 12 +- .../Parser/SerializedPlanParser.cpp | 9 +- .../Parser/SerializedPlanParser.h | 2 +- .../Parser/SparkRowToCHColumn.cpp | 10 +- .../local-engine/Parser/SparkRowToCHColumn.h | 13 +- .../aggregate_function_parser/CountParser.cpp | 3 +- .../LeadLagParser.cpp | 6 +- .../PercentileParserBase.cpp | 2 +- .../SimpleStatisticsFunctions.cpp | 3 +- .../arrayHighOrderFunctions.cpp | 46 ++- .../scalar_function_parser/arrayPosition.cpp | 2 +- .../Parser/scalar_function_parser/elt.cpp | 2 +- .../scalar_function_parser/findInset.cpp | 2 +- .../scalar_function_parser/getTimestamp.h | 2 +- .../Parser/scalar_function_parser/locate.cpp | 2 +- .../mapHighOrderFunctions.cpp | 2 +- .../Parser/scalar_function_parser/slice.cpp | 2 +- .../Storages/Cache/CacheManager.cpp | 4 +- .../IO/AggregateSerializationUtils.cpp | 2 +- .../local-engine/Storages/IO/NativeReader.cpp | 20 +- .../local-engine/Storages/IO/NativeWriter.cpp | 2 +- .../Storages/Kafka/GlutenKafkaSource.cpp | 3 +- .../MergeTree/MergeSparkMergeTreeTask.cpp | 3 +- .../Storages/MergeTree/MetaDataHelper.cpp | 18 +- .../Storages/MergeTree/SparkMergeTreeSink.cpp | 3 +- .../MergeTree/SparkMergeTreeWriter.cpp | 8 +- .../Storages/MergeTree/SparkMergeTreeWriter.h | 3 +- .../MergeTree/SparkStorageMergeTree.cpp | 44 ++- .../Storages/Parquet/ColumnIndexFilter.cpp | 106 +++---- .../Storages/Parquet/ColumnIndexFilter.h | 39 +-- .../Storages/Parquet/ParquetConverter.h | 4 +- .../Bitmap/DeltaDVRoaringBitmapArray.cpp | 2 +- .../Delta/Bitmap/DeltaDVRoaringBitmapArray.h | 2 +- .../SubstraitSource/Delta/DeltaWriter.cpp | 7 +- .../Iceberg/SimpleParquetReader.cpp | 2 +- .../SubstraitSource/ORCFormatFile.cpp | 4 +- .../SubstraitSource/ParquetFormatFile.cpp | 30 +- .../SubstraitSource/ReadBufferBuilder.cpp | 9 +- .../SubstraitSource/ReadBufferBuilder.h | 2 +- cpp-ch/local-engine/local_engine_jni.cpp | 6 +- .../apache/spark/sql/GlutenQueryTest.scala | 14 +- 104 files changed, 938 insertions(+), 448 deletions(-) create mode 100644 cpp-ch/local-engine/Functions/AggregateFunctionSparkArrayFold.cpp diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala index b4cefc3a77a2..0cd0f53aa8bc 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala @@ -464,14 +464,14 @@ class GlutenClickHouseTPCHSuite extends MergeTreeSuite { | insert into cross_join_t | select id as a, cast(id as string) as b, | concat('1231231232323232322', cast(id as string)) as c - | from range(0, 10000) + | from range(0, 5000) |""".stripMargin spark.sql(sql) sql = """ | insert into cross_join_t | select id as a, cast(id as string) as b, | concat('1231231232323232322', cast(id as string)) as c - | from range(10000, 20000) + | from range(5000, 10000) |""".stripMargin spark.sql(sql) sql = """ diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala index 88abe3ae2e05..400f66bf140f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala @@ -912,6 +912,59 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS val aggregate_sql = "select ids, aggregate(ids, 3, (acc, x) -> acc + x) from tb_array" runQueryAndCompare(aggregate_sql)(checkGlutenPlan[ProjectExecTransformer]) + + val aggregate_finish_sql = + """ + |select + | aggregate( + | ids, + | cast(struct(0 as count, 0.0 as sum) as struct), + | (acc, x) -> struct(acc.count + 1, acc.sum + coalesce(cast(x as double), 0.0)), + | acc -> acc.sum + | ) + |from tb_array + |""".stripMargin + runQueryAndCompare(aggregate_finish_sql)(checkGlutenPlan[ProjectExecTransformer]) + } + } + + test("array aggregate with nested struct and nulls") { + withTable("tb_array_complex") { + sql("create table tb_array_complex(items array>) using parquet") + sql(""" + |insert into tb_array_complex values + |(array(named_struct('v', 1, 'w', 1.5), named_struct('v', null, 'w', 2.0), null)), + |(array()), + |(null), + |(array(named_struct('v', 2, 'w', null), named_struct('v', 3, 'w', 4.5))) + |""".stripMargin) + + val aggregate_struct_sql = + """ + |select + | aggregate( + | items, + | cast(struct(0 as cnt, 0.0 as sum) as struct), + | (acc, x) -> struct( + | acc.cnt + if(x is null or x.v is null, 0, 1), + | acc.sum + coalesce(x.w, 0.0) + | ), + | acc -> if(acc.cnt = 0, cast(null as double), acc.sum / acc.cnt) + | ) as avg_w + |from tb_array_complex + |""".stripMargin + runQueryAndCompare(aggregate_struct_sql)(checkGlutenPlan[ProjectExecTransformer]) + + val transform_filter_sql = + """ + |select + | transform( + | filter(items, x -> x is not null), + | x -> coalesce(x.v, 0) + cast(coalesce(x.w, 0.0) as int) + | ) + |from tb_array_complex + |""".stripMargin + runQueryAndCompare(transform_filter_sql)(checkGlutenPlan[ProjectExecTransformer]) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenNothingValueCheck.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenNothingValueCheck.scala index 7bc1d2fbf978..bb265ddd2fc4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenNothingValueCheck.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenNothingValueCheck.scala @@ -175,14 +175,14 @@ class GlutenNothingValueCheck extends GlutenClickHouseWholeStageTransformerSuite test("nothing array in shuffle") { val sql = """ - |select t1.k1 as a, t1.k2 as b, t2.k1 as c, t2.k2 as d, t1.x as x from ( - | select k1, k2, array() as x from t1 + |select t1.k1 as a, t1.k2 as b, t2.k1 as c, t2.k2 as d, t1.x as x, t1.o as o from ( + | select k1, k2, array() as x, '' as o from t1 | union all - | select k1, k2, array(123) as x from t2 + | select k1, k2, array(123) as x, '123' as o from t2 |) t1 left join ( | select k1, k2 from t3 |) t2 on t1.k1 = t2.k1 and t1.k2 = t2.k2 - |order by t1.k1, t1.k2, t2.k1, t2.k2 + |order by t1.k1, t1.k2, t2.k1, t2.k2, t1.o |""".stripMargin compareResultsAgainstVanillaSpark(sql, true, { _ => }) } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeOptimizeSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeOptimizeSuite.scala index 9b260971090c..89f8a16f92e4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeOptimizeSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeOptimizeSuite.scala @@ -92,7 +92,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { val ret = spark.sql("select count(*) from lineitem_mergetree_optimize").collect() assertResult(600572)(ret.apply(0).get(0)) - assertResult(462)( + assertResult(480)( countFiles(new File(s"$dataHome/lineitem_mergetree_optimize")) ) // many merged parts } @@ -183,19 +183,19 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p2").collect() assertResult(600572)(ret.apply(0).get(0)) - assertResult(372)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p2"))) + assertResult(343)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p2"))) spark.sql("VACUUM lineitem_mergetree_optimize_p2 RETAIN 0 HOURS") if (spark32) { assertResult(239)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p2"))) } else { - assertResult(241)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p2"))) + assertResult(240)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p2"))) } spark.sql("VACUUM lineitem_mergetree_optimize_p2 RETAIN 0 HOURS") // the second VACUUM will remove some empty folders if (spark32) { assertResult(220)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p2"))) } else { - assertResult(226)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p2"))) + assertResult(229)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p2"))) } val ret2 = spark.sql("select count(*) from lineitem_mergetree_optimize_p2").collect() @@ -220,18 +220,18 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p3").collect() assertResult(600572)(ret.apply(0).get(0)) - assertResult(491)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p3"))) + assertResult(458)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p3"))) spark.sql("VACUUM lineitem_mergetree_optimize_p3 RETAIN 0 HOURS") if (spark32) { assertResult(302)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p3"))) } else { - assertResult(304)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p3"))) + assertResult(306)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p3"))) } spark.sql("VACUUM lineitem_mergetree_optimize_p3 RETAIN 0 HOURS") if (spark32) { assertResult(275)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p3"))) } else { - assertResult(281)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p3"))) + assertResult(288)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p3"))) } val ret2 = spark.sql("select count(*) from lineitem_mergetree_optimize_p3").collect() @@ -257,18 +257,18 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p4").collect() assertResult(600572)(ret.apply(0).get(0)) - assertResult(491)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p4"))) + assertResult(458)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p4"))) spark.sql("VACUUM lineitem_mergetree_optimize_p4 RETAIN 0 HOURS") if (spark32) { assertResult(302)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p4"))) } else { - assertResult(304)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p4"))) + assertResult(306)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p4"))) } spark.sql("VACUUM lineitem_mergetree_optimize_p4 RETAIN 0 HOURS") if (spark32) { assertResult(275)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p4"))) } else { - assertResult(281)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p4"))) + assertResult(288)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p4"))) } val ret2 = spark.sql("select count(*) from lineitem_mergetree_optimize_p4").collect() @@ -300,7 +300,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { } else { // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. // this case will create a checkpoint - assertResult(105)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p5"))) + assertResult(106)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p5"))) } val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p5").collect() @@ -323,7 +323,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { assertResult(93)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p5"))) } else { // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. - assertResult(104)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p5"))) + assertResult(106)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p5"))) } val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p5").collect() @@ -339,7 +339,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { assertResult(77)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p5"))) } else { // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. - assertResult(93)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p5"))) + assertResult(94)(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p5"))) } val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p5").collect() @@ -367,11 +367,11 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { assertResult(600572)(ret.apply(0).get(0)) assertResult(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p6")))( - if (spark32) 491 else 519) + if (spark32) 491 else 489) spark.sql("VACUUM lineitem_mergetree_optimize_p6 RETAIN 0 HOURS") spark.sql("VACUUM lineitem_mergetree_optimize_p6 RETAIN 0 HOURS") assertResult(countFiles(new File(s"$dataHome/lineitem_mergetree_optimize_p6")))( - if (spark32) 315 else 327) + if (spark32) 315 else 333) val ret2 = spark.sql("select count(*) from lineitem_mergetree_optimize_p6").collect() assertResult(600572)(ret2.apply(0).get(0)) @@ -443,7 +443,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { if (spark32) { assertResult(99)(countFiles(new File(dataPath))) } else { - assertResult(105)(countFiles(new File(dataPath))) + assertResult(106)(countFiles(new File(dataPath))) } val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() @@ -466,7 +466,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { if (spark32) { assertResult(93)(countFiles(new File(dataPath))) } else { - assertResult(104)(countFiles(new File(dataPath))) + assertResult(106)(countFiles(new File(dataPath))) } val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() @@ -482,7 +482,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite extends CreateMergeTreeSuite { if (spark32) { assertResult(77)(countFiles(new File(dataPath))) } else { - assertResult(93)(countFiles(new File(dataPath))) + assertResult(94)(countFiles(new File(dataPath))) } val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala index c112a3b7797c..d5fcde06d325 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala @@ -99,6 +99,35 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite extends ParquetTPCHSuit } } + test("TPCH Q6 + New CH Parquet V3") { + compareResultsAgainstVanillaSpark( + """ + |SELECT + | sum(l_extendedprice * l_discount) AS revenue + |FROM + | lineitem + |WHERE + | l_shipdate >= date'1994-01-01' + | AND l_shipdate < date'1994-01-01' + interval 1 year + | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 + | AND l_orderkey = 119430; + |""".stripMargin, + customCheck = { + df => + assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + } + assert(plans.size == 1) + + assert(plans(0).metrics("numFiles").value === 1) + assert(plans(0).metrics("pruningTime").value === pruningTimeValueSpark) + assert(plans(0).metrics("filesSize").value === 19230111) + assert(plans(0).metrics("numOutputRows").value === 20000) + } + ) + } + test("Check the metrics values") { withSQLConf((GlutenConfig.COLUMNAR_SORT_ENABLED.key, "false")) { customCheck(1, native = false) { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala index 0222bdfd0f7b..bd168fa5950b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala @@ -268,7 +268,7 @@ class GlutenClickHouseTPCHParquetBucketSuite assert(!plans.head.asInstanceOf[FileSourceScanExecTransformer].bucketedScan) assert(plans.head.metrics("numFiles").value === 4) assert(plans.head.metrics("pruningTime").value === pruningTimeValueSpark) - assert(plans.head.metrics("numOutputRows").value === 600572) + assert(plans.head.metrics("numOutputRows").value === 160000) } } @@ -297,7 +297,7 @@ class GlutenClickHouseTPCHParquetBucketSuite assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) assert(plans(2).metrics("numFiles").value === 4) - assert(plans(2).metrics("numOutputRows").value === 600572) + assert(plans(2).metrics("numOutputRows").value === 240000) } withSQLConf( diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 614c69544f2d..ac8ea9d696e1 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20250916 -CH_COMMIT=39da31eab7b +CH_BRANCH=rebase_ch/20260310_25_12 +CH_COMMIT=b7393160b95 diff --git a/cpp-ch/local-engine/CMakeLists.txt b/cpp-ch/local-engine/CMakeLists.txt index 459d038c532a..2cd14ba2efe5 100644 --- a/cpp-ch/local-engine/CMakeLists.txt +++ b/cpp-ch/local-engine/CMakeLists.txt @@ -88,7 +88,8 @@ include_directories( ${ClickHouse_SOURCE_DIR}/contrib/llvm-project/llvm/include ${ClickHouse_SOURCE_DIR}/contrib/llvm-project/utils/bazel/llvm-project-overlay/llvm/include ${ClickHouse_SOURCE_DIR}/contrib/libdivide - ${ClickHouse_SOURCE_DIR}/contrib/libdivide-cmake) + ${ClickHouse_SOURCE_DIR}/contrib/libdivide-cmake + ${ClickHouse_SOURCE_DIR}/contrib/StringZilla/include) add_subdirectory(Storages/Parquet) add_subdirectory(Storages/SubstraitSource) diff --git a/cpp-ch/local-engine/Common/AggregateUtil.cpp b/cpp-ch/local-engine/Common/AggregateUtil.cpp index f2f23c13efb7..f2c7d7e8817c 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.cpp +++ b/cpp-ch/local-engine/Common/AggregateUtil.cpp @@ -52,6 +52,8 @@ extern const SettingsUInt64 max_block_size; extern const SettingsBool compile_aggregate_expressions; extern const SettingsUInt64 min_count_to_compile_aggregate_expression; extern const SettingsBool enable_software_prefetch_in_aggregation; +extern const SettingsBool enable_producing_buckets_out_of_order_in_aggregation; +extern const SettingsBool serialize_string_in_memory_with_zero_byte; } template @@ -259,7 +261,9 @@ DB::Aggregator::Params AggregatorParamsHelper::buildParams( only_merge, aggregate_settings[DB::Setting::optimize_group_by_constant_keys], aggregate_settings[DB::Setting::min_hit_rate_to_use_consecutive_keys_optimization], - {}}; + {}, + settings[DB::Setting::enable_producing_buckets_out_of_order_in_aggregation], + settings[DB::Setting::serialize_string_in_memory_with_zero_byte]}; } diff --git a/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp b/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp index aa88b42db1c6..20240fa5b758 100644 --- a/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp +++ b/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -39,6 +40,7 @@ extern const int LOGICAL_ERROR; namespace Setting { extern const SettingsUInt64 max_block_size; +extern const SettingsBool enable_lazy_columns_replication; } } @@ -146,7 +148,7 @@ addArrayJoinStep(DB::ContextPtr context, DB::QueryPlan & plan, const DB::Actions array_join.columns = std::move(array_joined_columns); array_join.is_left = is_left; auto array_join_step = std::make_unique( - plan.getCurrentHeader(), std::move(array_join), false, context->getSettingsRef()[DB::Setting::max_block_size]); + plan.getCurrentHeader(), std::move(array_join), false, context->getSettingsRef()[DB::Setting::max_block_size], context->getSettingsRef()[DB::Setting::enable_lazy_columns_replication]); array_join_step->setStepDescription("ARRAY JOIN In Generate"); steps.emplace_back(array_join_step.get()); plan.addStep(std::move(array_join_step)); diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index a7c0df4777f4..67cfb5545fe2 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -409,6 +409,7 @@ const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeType( DB::ActionsDAG & actions_dag, const DB::ActionsDAG::Node * node, const DB::DataTypePtr & cast_to_type, + DB::ContextPtr context, const std::string & result_name, DB::CastType cast_type) { @@ -421,7 +422,7 @@ const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeType( DB::CastDiagnostic diagnostic = {node->result_name, node->result_name}; DB::ColumnWithTypeAndName left_column{nullptr, node->result_type, {}}; DB::ActionsDAG::NodeRawConstPtrs children = {left_arg, right_arg}; - auto func_base_cast = createInternalCast(std::move(left_column), cast_to_type, cast_type, diagnostic); + auto func_base_cast = createInternalCast(std::move(left_column), cast_to_type, cast_type, diagnostic, context); return &actions_dag.addFunction(func_base_cast, std::move(children), result_name); } @@ -430,13 +431,14 @@ const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeTypeIfNeeded( DB::ActionsDAG & actions_dag, const DB::ActionsDAG::Node * node, const DB::DataTypePtr & dst_type, + DB::ContextPtr context, const std::string & result_name, DB::CastType cast_type) { if (node->result_type->equals(*dst_type)) return node; - return convertNodeType(actions_dag, node, dst_type, result_name, cast_type); + return convertNodeType(actions_dag, node, dst_type, context, result_name, cast_type); } String QueryPipelineUtil::explainPipeline(DB::QueryPipeline & pipeline) @@ -791,6 +793,8 @@ void BackendInitializerUtil::initContexts(DB::Context::ConfigurationPtr config) }; global_context->setTemporaryStoragePath(tmp_path, 0); + if (!fs::exists(tmp_path)) + fs::create_directories(tmp_path); global_context->setPath(config->getString("path", "/")); String uncompressed_cache_policy = config->getString("uncompressed_cache_policy", DEFAULT_UNCOMPRESSED_CACHE_POLICY); diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index 709af367ea9c..614779081469 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -122,6 +122,7 @@ class ActionsDAGUtil DB::ActionsDAG & actions_dag, const DB::ActionsDAG::Node * node_to_cast, const DB::DataTypePtr & cast_to_type, + DB::ContextPtr context, const std::string & result_name = "", DB::CastType cast_type = DB::CastType::nonAccurate); @@ -129,6 +130,7 @@ class ActionsDAGUtil DB::ActionsDAG & actions_dag, const DB::ActionsDAG::Node * node, const DB::DataTypePtr & dst_type, + DB::ContextPtr context, const std::string & result_name = "", DB::CastType cast_type = DB::CastType::nonAccurate); }; diff --git a/cpp-ch/local-engine/Common/DebugUtils.cpp b/cpp-ch/local-engine/Common/DebugUtils.cpp index 1dea6f508f5d..20c926bdc367 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.cpp +++ b/cpp-ch/local-engine/Common/DebugUtils.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include @@ -159,7 +158,7 @@ static std::string toString(const DB::IColumn * const col, size_t row, size_t wi if (col->isNullAt(row)) return "null"; - std::string str = DB::toString((*col)[row]); + std::string str = DB::fieldToString((*col)[row]); if (str.size() <= width) return str; return str.substr(0, width - 3) + "..."; @@ -434,7 +433,7 @@ std::string dumpColumn(const std::string & name, const DB::ColumnPtr & column) //TODO: ColumnSet if (isColumnConst(*column)) - return toString(assert_cast(*column).getField()); + return DB::fieldToString(assert_cast(*column).getField()); size_t size = std::min(static_cast(10), column->size()); std::vector results; diff --git a/cpp-ch/local-engine/Common/PlanUtil.cpp b/cpp-ch/local-engine/Common/PlanUtil.cpp index 1a0eb83ec42f..475e292fc453 100644 --- a/cpp-ch/local-engine/Common/PlanUtil.cpp +++ b/cpp-ch/local-engine/Common/PlanUtil.cpp @@ -15,20 +15,30 @@ * limitations under the License. */ #include "PlanUtil.h" +#include #include #include #include #include #include +#include #include #include #include #include -namespace DB::ErrorCodes +namespace DB { -extern const int UNKNOWN_TYPE; +namespace ErrorCodes +{ + extern const int UNKNOWN_TYPE; +} + +namespace Setting +{ +extern const SettingsUInt64 query_plan_max_step_description_length; +} } namespace local_engine::PlanUtil @@ -79,14 +89,15 @@ void checkOuputType(const DB::QueryPlan & plan) } } -DB::IQueryPlanStep * adjustQueryPlanHeader(DB::QueryPlan & plan, const DB::Block & to_header, const String & step_desc) +DB::IQueryPlanStep * adjustQueryPlanHeader(DB::QueryPlan & plan, const DB::Block & to_header, const String & step_desc, const DB::ContextPtr context) { auto convert_actions_dag = DB::ActionsDAG::makeConvertingActions( plan.getCurrentHeader()->getColumnsWithTypeAndName(), to_header.getColumnsWithTypeAndName(), - DB::ActionsDAG::MatchColumnsMode::Name); + DB::ActionsDAG::MatchColumnsMode::Name, + context); auto expression_step = std::make_unique(plan.getCurrentHeader(), std::move(convert_actions_dag)); - expression_step->setStepDescription(step_desc); + expression_step->setStepDescription(step_desc, context->getSettingsRef()[DB::Setting::query_plan_max_step_description_length]); plan.addStep(std::move(expression_step)); return plan.getRootNode()->step.get(); } @@ -119,7 +130,7 @@ DB::IQueryPlanStep * renamePlanHeader(DB::QueryPlan & plan, const BuildNamesWith buildAliases(*plan.getCurrentHeader(), aliases); actions_dag.project(aliases); auto expression_step = std::make_unique(plan.getCurrentHeader(), std::move(actions_dag)); - expression_step->setStepDescription(step_desc); + expression_step->setStepDescription(step_desc, 1000); plan.addStep(std::move(expression_step)); return plan.getRootNode()->step.get(); } diff --git a/cpp-ch/local-engine/Common/PlanUtil.h b/cpp-ch/local-engine/Common/PlanUtil.h index 9b91bb8e8c5d..e0aeb243711a 100644 --- a/cpp-ch/local-engine/Common/PlanUtil.h +++ b/cpp-ch/local-engine/Common/PlanUtil.h @@ -38,7 +38,7 @@ using BuildNamesWithAliases = const std::function & columns); DB::IQueryPlanStep * renamePlanHeader(DB::QueryPlan & plan, const BuildNamesWithAliases & buildAliases, const std::string & step_desc = "Rename Output"); diff --git a/cpp-ch/local-engine/Common/QueryContext.cpp b/cpp-ch/local-engine/Common/QueryContext.cpp index c6aaa9ccc5d3..3eb841eafa3b 100644 --- a/cpp-ch/local-engine/Common/QueryContext.cpp +++ b/cpp-ch/local-engine/Common/QueryContext.cpp @@ -19,7 +19,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -35,6 +37,11 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; } + +namespace Setting +{ +extern const SettingsInt32 os_threads_nice_value_query; +} } using namespace DB; @@ -118,10 +125,10 @@ int64_t QueryContext::initializeQuery(const String & task_id) // // Notice: // this generated random query id a qualified global queryid for the spark query - query_context->query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4()) + "_" + task_id); + query_context->query_context->setCurrentQueryId(DB::toString(UUIDHelpers::generateV4()) + "_" + task_id); auto config = MemoryConfig::loadFromContext(query_context->query_context); query_context->thread_status = std::make_shared(); - query_context->thread_group = std::make_shared(query_context->query_context); + query_context->thread_group = std::make_shared(query_context->query_context, query_context->query_context->getSettingsRef()[Setting::os_threads_nice_value_query]); CurrentThread::attachToGroup(query_context->thread_group); auto memory_limit = config.off_heap_per_task; @@ -167,7 +174,7 @@ void QueryContext::logCurrentPerformanceCounters(ProfileEvents::Counters & count msg << "\n---------------------Task Performance Counters(" << task_id << ")-----------------------------\n"; for (ProfileEvents::Event event = ProfileEvents::Event(0); event < counters.num_counters; event++) { - const auto * name = ProfileEvents::getName(event); + const auto name = ProfileEvents::getName(event); const auto * doc = ProfileEvents::getDocumentation(event); auto & count = counters[event]; if (count == 0) diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp index 1b5ff914aa19..c4ac1d6fe886 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp @@ -65,8 +65,8 @@ void CompactObjectStorageDiskTransaction::commit(const DB::TransactionCommitOpti std::filesystem::path meta_path = std::filesystem::path(prefix_path) / PART_META_FILE_NAME; auto object_storage = disk.getObjectStorage(); - auto data_key = object_storage->generateObjectKeyForPath(data_path, std::nullopt); - auto meta_key = object_storage->generateObjectKeyForPath(meta_path, std::nullopt); + auto data_key = metadata_tx->generateObjectKeyForPath(data_path); + auto meta_key = metadata_tx->generateObjectKeyForPath(meta_path); disk.createDirectories(prefix_path); auto data_write_buffer = object_storage->writeObject(DB::StoredObject(data_key.serialize(), data_path), DB::WriteMode::Rewrite); @@ -89,15 +89,15 @@ void CompactObjectStorageDiskTransaction::commit(const DB::TransactionCommitOpti file_size += count; out.write(buffer.data(), count); } - metadata.addObject(key, offset, file_size); - metadata_tx->writeStringToFile(item.first, metadata.serializeToString()); + metadata.objects.emplace_back(key.serialize(), "", file_size, offset); + metadata_tx->writeStringToFile(item.first, metadata.serializeToStringWithOffset()); offset += file_size; }); // You can load the complete file in advance through this metadata original, which improves the download efficiency of mergetree metadata. DB::DiskObjectStorageMetadata whole_meta(object_storage->getCommonKeyPrefix(), local_path); - whole_meta.addObject(key, 0, offset); - metadata_tx->writeStringToFile(local_path, whole_meta.serializeToString()); + whole_meta.objects.emplace_back(key.serialize(), "", offset, 0); + metadata_tx->writeStringToFile(local_path, whole_meta.serializeToStringWithOffset()); out.sync(); out.finalize(); }; @@ -109,12 +109,20 @@ void CompactObjectStorageDiskTransaction::commit(const DB::TransactionCommitOpti files.clear(); } +std::unique_ptr CompactObjectStorageDiskTransaction::writeFileWithAutoCommit( + const std::string & path, + size_t buf_size, + DB::WriteMode mode, + const DB::WriteSettings & settings) +{ + return writeFile(path, buf_size, mode, settings); +} + std::unique_ptr CompactObjectStorageDiskTransaction::writeFile( const std::string & path, size_t buf_size, DB::WriteMode mode, - const DB::WriteSettings &, - bool) + const DB::WriteSettings &) { if (mode != DB::WriteMode::Rewrite) { @@ -128,11 +136,11 @@ std::unique_ptr CompactObjectStorageDiskTransaction "Don't support write file in different dirs, path {}, prefix path: {}", path, prefix_path); - auto tmp = std::make_shared(tmp_data.get()); + auto tmp = std::make_shared(tmp_data); files.emplace_back(path, tmp); auto tx = disk.getMetadataStorage()->createTransaction(); tx->createDirectoryRecursive(std::filesystem::path(path).parent_path()); - tx->createEmptyMetadataFile(path); + tx->createMetadataFile(path, {}); tx->commit(); return std::make_unique(path, tmp); diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h index 721989890785..858e8ce13855 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h @@ -16,8 +16,8 @@ */ #pragma once #include -#include -#include +#include +#include #include @@ -71,7 +71,7 @@ class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { void commit(const DB::TransactionCommitOptionsVariant & options) override; - void undo() override + void undo() noexcept override { throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Operation `undo` is not implemented"); } @@ -91,11 +91,6 @@ class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { disk.createFile(path); } - void clearDirectory(const std::string & path) override - { - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Operation `clearDirectory` is not implemented"); - } - void moveDirectory(const std::string & from_path, const std::string & to_path) override { throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Operation `moveDirectory` is not implemented"); @@ -116,12 +111,17 @@ class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Operation `copyFile` is not implemented"); } + std::unique_ptr writeFileWithAutoCommit( + const std::string & path, + size_t buf_size, + DB::WriteMode mode, + const DB::WriteSettings & settings) override; + std::unique_ptr writeFile( /// NOLINT const std::string & path, size_t buf_size, DB::WriteMode mode, - const DB::WriteSettings & settings, - bool /*autocommit */) override; + const DB::WriteSettings & settings) override; void writeFileUsingBlobWritingFunction(const String & path, DB::WriteMode mode, WriteBlobFunction && write_blob_function) override diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.h b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.h index 33c85f9c3911..51dc01eef5f6 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.h @@ -19,8 +19,8 @@ #include -#include -#include +#include +#include #include #include #include diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp index eafcd636f788..a72f38b65cb4 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp @@ -19,7 +19,7 @@ #include "GlutenDiskS3.h" #include -#include +#include #include #include diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.h b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.h index 0774e0cd3c1f..495ce17ae28b 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.h @@ -18,7 +18,7 @@ #pragma once -#include +#include #if USE_AWS_S3 diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.cpp index 6ea76ca0f06e..ee6910ba35cf 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.cpp @@ -24,7 +24,6 @@ namespace local_engine std::unique_ptr GlutenHDFSObjectStorage::readObject( /// NOLINT const StoredObject & object, const ReadSettings & read_settings, - std::optional, std::optional) const { size_t begin_of_path = object.remote_path.find('/', object.remote_path.find("//") + 2); @@ -39,12 +38,5 @@ std::unique_ptr GlutenHDFSObjectStorage::readObject( /// read_settings.remote_read_buffer_use_external_buffer); } -DB::ObjectStorageKey local_engine::GlutenHDFSObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const -{ - initializeHDFSFS(); - /// what ever data_source_description.description value is, consider that key as relative key - chassert(data_directory.starts_with("/")); - return ObjectStorageKey::createAsRelative(fs::path(url_without_path) / data_directory.substr(1), path); -} } #endif diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.h b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.h index 82f3003eb3d7..771b3ec0267e 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.h @@ -19,7 +19,7 @@ #include "config.h" #if USE_HDFS -#include +#include #endif namespace local_engine @@ -39,9 +39,7 @@ class GlutenHDFSObjectStorage final : public DB::HDFSObjectStorage std::unique_ptr readObject( /// NOLINT const DB::StoredObject & object, const DB::ReadSettings & read_settings = DB::ReadSettings{}, - std::optional read_hint = {}, - std::optional file_size = {}) const override; - DB::ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; + std::optional read_hint = {}) const override; hdfsFS getHDFSFS() const { return hdfs_fs.get(); } }; #endif diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.cpp index 212a998ddbd9..0523f85ae324 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.cpp @@ -16,9 +16,9 @@ */ #include "MetadataStorageFromRocksDB.h" #if USE_ROCKSDB -#include +#include #include -#include +#include #include #include #include @@ -59,8 +59,9 @@ MetadataStorageFromRocksDB::MetadataStorageFromRocksDB( rocksdb::Options options; options.create_if_missing = true; throwRockDBErrorNotOk(rocksdb::DB::Open(options, rocksdb_dir, &rocksdb)); + // TODO: rebase-25.12, is it correct to use createEmpty storageID? metadata_clean_task = QueryContext::globalContext()->getSchedulePool().createTask( - "MetadataStorageFromRocksDB", [this] { cleanOutdatedMetadataThreadFunc(); }); + DB::StorageID::createEmpty(), "MetadataStorageFromRocksDB", [this] { cleanOutdatedMetadataThreadFunc(); }); metadata_clean_task->scheduleAfter(metadata_clean_task_interval_seconds * 1000); logger = getLogger("MetadataStorageFromRocksDB"); } @@ -99,7 +100,7 @@ bool MetadataStorageFromRocksDB::existsDirectory(const std::string & path) const uint64_t MetadataStorageFromRocksDB::getFileSize(const std::string & path) const { - return readMetadata(path)->getTotalSizeBytes(); + return getTotalSize(readMetadata(path)->objects); } Poco::Timestamp MetadataStorageFromRocksDB::getLastModified(const std::string & /*path*/) const @@ -144,15 +145,7 @@ uint32_t MetadataStorageFromRocksDB::getHardlinkCount(const std::string & /*path DB::StoredObjects MetadataStorageFromRocksDB::getStorageObjects(const std::string & path) const { - auto metadata = readMetadata(path); - const auto & keys_with_meta = metadata->getKeysWithMeta(); - - DB::StoredObjects objects; - objects.reserve(keys_with_meta.size()); - for (const auto & [object_key, object_meta] : keys_with_meta) - objects.emplace_back(object_key.serialize(), path, object_meta.size_bytes, object_meta.offset); - - return objects; + return readMetadata(path)->objects; } DB::DiskObjectStorageMetadataPtr MetadataStorageFromRocksDB::readMetadata(const std::string & path) const @@ -259,7 +252,8 @@ rocksdb::DB & MetadataStorageFromRocksDB::getRocksDB() const void MetadataStorageFromRocksDBTransaction::commit(const DB::TransactionCommitOptionsVariant & options) { - commitImpl(options, metadata_storage.getMetadataMutex()); + // TODO: rebase-25.12, fix later, is it OK to call 'DB::MetadataOperationsHolder::commit()'? + DB::MetadataOperationsHolder::commit(); } std::optional MetadataStorageFromRocksDBTransaction::tryGetBlobsFromTransactionIfExists(const std::string & path) const @@ -286,9 +280,9 @@ void MetadataStorageFromRocksDBTransaction::createEmptyMetadataFile(const std::s void MetadataStorageFromRocksDBTransaction::createMetadataFile(const std::string & path, DB::ObjectStorageKey key, uint64_t size_in_bytes) { auto metadata = std::make_unique(metadata_storage.compatible_key_prefix, path); - metadata->addObject(std::move(key), size_in_bytes); + metadata->objects.emplace_back(key.serialize(), "", size_in_bytes, 0); - auto data = metadata->serializeToString(); + auto data = metadata->serializeToStringWithOffset(); if (!data.empty()) addOperation(std::make_unique(path, metadata_storage.getRocksDB(), data)); } @@ -322,5 +316,18 @@ void MetadataStorageFromRocksDBTransaction::unlinkFile(const std::string & path) { addOperation(std::make_unique(path, metadata_storage.getRocksDB())); } + +DB::ObjectStorageKey MetadataStorageFromRocksDBTransaction::generateObjectKeyForPath(const std::string & path) +{ + // TODO: rebase-25.12, how to return ObjectStorageKey + // return metadata_storage.key_generator->generate(); + return DB::ObjectStorageKey(); +} + +void MetadataStorageFromRocksDBTransaction::createMetadataFile(const std::string & path, const DB::StoredObjects & objects) +{ + // TODO: rebase-25.12, fix later, there is no operations + // operations.addOperation(std::make_unique(path, objects, metadata_storage.compatible_key_prefix, *metadata_storage.disk)); +} } #endif \ No newline at end of file diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.h b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.h index 1609cf75f8bf..c9ad0d0b050d 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.h @@ -20,9 +20,9 @@ #include #include #include -#include -#include -#include +#include +#include +#include #include namespace local_engine @@ -48,7 +48,8 @@ class MetadataStorageFromRocksDB final : public DB::IMetadataStorage Poco::Timestamp getLastModified(const std::string & path) const override; bool supportsChmod() const override; bool supportsStat() const override; - bool supportsPartitionCommand(const DB::PartitionCommand & command) const override; + // TODO: rebase-25.12, remove override first, fix later + bool supportsPartitionCommand(const DB::PartitionCommand & command) const; std::vector listDirectory(const std::string & path) const override; DB::DirectoryIteratorPtr iterateDirectory(const std::string & path) const override; uint32_t getHardlinkCount(const std::string & path) const override; @@ -59,6 +60,7 @@ class MetadataStorageFromRocksDB final : public DB::IMetadataStorage std::string readFileToString(const std::string & path) const override; void shutdown() override; void cleanOutdatedMetadataThreadFunc(); + bool areBlobPathsRandom() const override { return false; } private: DB::SharedMutex & getMetadataMutex() const; @@ -87,8 +89,10 @@ class MetadataStorageFromRocksDBTransaction final : public DB::IMetadataTransact void commit(const DB::TransactionCommitOptionsVariant & options) override; const DB::IMetadataStorage & getStorageForNonTransactionalReads() const override; bool supportsChmod() const override; - void createEmptyMetadataFile(const std::string & path) override; - void createMetadataFile(const std::string & path, DB::ObjectStorageKey key, uint64_t size_in_bytes) override; + // TODO: rebase-25.12, remove override first, fix later + void createEmptyMetadataFile(const std::string & path); + // TODO: rebase-25.12, remove override first, fix later + void createMetadataFile(const std::string & path, DB::ObjectStorageKey key, uint64_t size_in_bytes); void writeStringToFile(const std::string &, const std::string &) override; void createDirectory(const std::string &) override; @@ -97,6 +101,8 @@ class MetadataStorageFromRocksDBTransaction final : public DB::IMetadataTransact void removeRecursive(const std::string &) override; void unlinkFile(const std::string &) override; std::optional tryGetBlobsFromTransactionIfExists(const std::string &) const override; + DB::ObjectStorageKey generateObjectKeyForPath(const std::string & path) override; + void createMetadataFile(const std::string & path, const DB::StoredObjects & objects) override; private: const MetadataStorageFromRocksDB & metadata_storage; diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDBTransactionOperations.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDBTransactionOperations.cpp index 0219dd2629bf..9a27036719bb 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDBTransactionOperations.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDBTransactionOperations.cpp @@ -72,7 +72,7 @@ std::vector listKeys(rocksdb::DB & db, const std::string & path) return result; } -void RocksDBWriteFileOperation::execute(std::unique_lock &) +void RocksDBWriteFileOperation::execute() { auto status = db.Get({}, path, &prev_data); if (status.IsNotFound()) @@ -82,7 +82,7 @@ void RocksDBWriteFileOperation::execute(std::unique_lock &) db.Put({}, path, data); } -void RocksDBWriteFileOperation::undo(std::unique_lock &) +void RocksDBWriteFileOperation::undo() { if (existed) throwRockDBErrorNotOk(db.Put({}, path, prev_data)); @@ -90,7 +90,7 @@ void RocksDBWriteFileOperation::undo(std::unique_lock &) throwRockDBErrorNotOk(db.Delete({}, path)); } -void RocksDBCreateDirectoryOperation::execute(std::unique_lock &) +void RocksDBCreateDirectoryOperation::execute() { existed = exist(db, path); if (existed) @@ -98,13 +98,13 @@ void RocksDBCreateDirectoryOperation::execute(std::unique_lock throwRockDBErrorNotOk(db.Put({}, path, DIR_DATA)); } -void RocksDBCreateDirectoryOperation::undo(std::unique_lock &) +void RocksDBCreateDirectoryOperation::undo() { if (existed) return; throwRockDBErrorNotOk(db.Delete({}, path)); } -void RocksDBCreateDirectoryRecursiveOperation::execute(std::unique_lock & ) +void RocksDBCreateDirectoryRecursiveOperation::execute( ) { namespace fs = std::filesystem; fs::path p(path); @@ -119,13 +119,13 @@ void RocksDBCreateDirectoryRecursiveOperation::execute(std::unique_lock & ) +void RocksDBCreateDirectoryRecursiveOperation::undo( ) { for (const auto & path_created : paths_created) throwRockDBErrorNotOk(db.Delete({}, path_created)); } -void RocksDBRemoveDirectoryOperation::execute(std::unique_lock &) +void RocksDBRemoveDirectoryOperation::execute() { auto *it = db.NewIterator({}); bool empty_dir = true; @@ -147,13 +147,13 @@ void RocksDBRemoveDirectoryOperation::execute(std::unique_lock throwRockDBErrorNotOk(db.Delete({}, path)); } -void RocksDBRemoveDirectoryOperation::undo(std::unique_lock &) +void RocksDBRemoveDirectoryOperation::undo() { if (existed) throwRockDBErrorNotOk(db.Put({}, path, RocksDBCreateDirectoryOperation::DIR_DATA)); } -void RocksDBRemoveRecursiveOperation::execute(std::unique_lock &) +void RocksDBRemoveRecursiveOperation::execute() { auto *it = db.NewIterator({}); for (it->Seek(path); it->Valid() && it->key().starts_with(path); it->Next()) @@ -163,19 +163,19 @@ void RocksDBRemoveRecursiveOperation::execute(std::unique_lock } } -void RocksDBRemoveRecursiveOperation::undo(std::unique_lock &) +void RocksDBRemoveRecursiveOperation::undo() { for (const auto & [key, value] : files) throwRockDBErrorNotOk(db.Put({}, key, value)); } -void RocksDBUnlinkFileOperation::execute(std::unique_lock &) +void RocksDBUnlinkFileOperation::execute() { prev_data = getData(db, path); throwRockDBErrorNotOk(db.Delete({}, path)); } -void RocksDBUnlinkFileOperation::undo(std::unique_lock &) +void RocksDBUnlinkFileOperation::undo() { throwRockDBErrorNotOk(db.Put({}, path, prev_data)); } diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDBTransactionOperations.h b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDBTransactionOperations.h index a6c344d84285..dcb8e4849ed7 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDBTransactionOperations.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDBTransactionOperations.h @@ -17,8 +17,8 @@ #pragma once #include #if USE_ROCKSDB -#include -#include +#include +#include #include namespace local_engine @@ -36,9 +36,9 @@ struct RocksDBWriteFileOperation final : public DB::IMetadataOperation { } - void execute(std::unique_lock & metadata_lock) override; + void execute() override; - void undo(std::unique_lock & metadata_lock) override; + void undo() override; private: std::string path; @@ -54,9 +54,9 @@ struct RocksDBCreateDirectoryOperation final : public DB::IMetadataOperation { } - void execute(std::unique_lock & metadata_lock) override; + void execute() override; - void undo(std::unique_lock & metadata_lock) override; + void undo() override; const static inline String DIR_DATA = "__DIR__"; private: @@ -72,9 +72,9 @@ struct RocksDBCreateDirectoryRecursiveOperation final : public DB::IMetadataOper { }; - void execute(std::unique_lock & metadata_lock) override; + void execute() override; - void undo(std::unique_lock & metadata_lock) override; + void undo() override; private: std::string path; @@ -88,9 +88,9 @@ struct RocksDBRemoveDirectoryOperation final : public DB::IMetadataOperation { } - void execute(std::unique_lock & metadata_lock) override; + void execute() override; - void undo(std::unique_lock & metadata_lock) override; + void undo() override; private: std::string path; @@ -104,9 +104,9 @@ struct RocksDBRemoveRecursiveOperation final : public DB::IMetadataOperation { } - void execute(std::unique_lock & metadata_lock) override; + void execute() override; - void undo(std::unique_lock & metadata_lock) override; + void undo() override; private: std::string path; @@ -120,9 +120,9 @@ struct RocksDBUnlinkFileOperation final : public DB::IMetadataOperation { } - void execute(std::unique_lock & metadata_lock) override; + void execute() override; - void undo(std::unique_lock & metadata_lock) override; + void undo() override; private: std::string path; diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/registerGlutenDiskObjectStorage.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/registerGlutenDiskObjectStorage.cpp index e57c31b45e24..70cdadff5aae 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/registerGlutenDiskObjectStorage.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/registerGlutenDiskObjectStorage.cpp @@ -17,14 +17,15 @@ #include "config.h" #include -#include +#include #include #include #include +#include #if USE_AWS_S3 -#include -#include +#include +#include #endif #if USE_HDFS @@ -84,7 +85,7 @@ void registerGlutenS3ObjectStorage(ObjectStorageFactory & factory) auto settings = std::make_unique(); settings->loadFromConfigForObjectStorage(config, config_prefix, context->getSettingsRef(), uri.uri.getScheme(), true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */ true); - auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(uri.key); + auto key_generator = DB::createObjectStorageKeyGeneratorByPrefix(uri.key); auto object_storage = std::make_shared(std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); diff --git a/cpp-ch/local-engine/Disks/registerGlutenDisks.cpp b/cpp-ch/local-engine/Disks/registerGlutenDisks.cpp index 2edf5b167996..a49ac366a84b 100644 --- a/cpp-ch/local-engine/Disks/registerGlutenDisks.cpp +++ b/cpp-ch/local-engine/Disks/registerGlutenDisks.cpp @@ -18,9 +18,9 @@ #include "config.h" #include -#include -#include -#include +#include +#include +#include #include #if USE_HDFS diff --git a/cpp-ch/local-engine/Functions/AggregateFunctionSparkArrayFold.cpp b/cpp-ch/local-engine/Functions/AggregateFunctionSparkArrayFold.cpp new file mode 100644 index 000000000000..a2e4af234819 --- /dev/null +++ b/cpp-ch/local-engine/Functions/AggregateFunctionSparkArrayFold.cpp @@ -0,0 +1,297 @@ +/* + * 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. + */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; +} +} + +namespace local_engine +{ +using namespace DB; + +class SparkFunctionArrayFold : public IFunction +{ +public: + static constexpr auto name = "sparkArrayFold"; + static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared(); } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; } + + void getLambdaArgumentTypes(DataTypes & arguments) const override + { + if (arguments.size() != 3 && arguments.size() != 4) + throw Exception( + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, + "Function {} requires a lambda function, an array, and an initial value, with an optional finish lambda", + getName()); + + const auto * merge_lambda = checkAndGetDataType(arguments[0].get()); + if (!merge_lambda || merge_lambda->getArgumentTypes().size() != 2) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument of function {} must be a lambda function with 2 arguments, found {} instead.", + getName(), + arguments[0]->getName()); + + const auto * array_type = checkAndGetDataType(removeNullable(arguments[1]).get()); + if (!array_type) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Second argument of function {} must be an array, found {} instead.", + getName(), + arguments[1]->getName()); + + DataTypes merge_lambda_args = {arguments[2], array_type->getNestedType()}; + arguments[0] = std::make_shared(merge_lambda_args, merge_lambda->getReturnType()); + + if (arguments.size() == 4) + { + const auto * finish_lambda = checkAndGetDataType(arguments[3].get()); + if (!finish_lambda || finish_lambda->getArgumentTypes().size() != 1) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Fourth argument of function {} must be a lambda function with 1 argument, found {} instead.", + getName(), + arguments[3]->getName()); + + DataTypes finish_lambda_args = {arguments[2]}; + arguments[3] = std::make_shared(finish_lambda_args, finish_lambda->getReturnType()); + } + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 3 && arguments.size() != 4) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires 3 or 4 arguments", getName()); + + const auto * array_type = checkAndGetDataType(removeNullable(arguments[1].type).get()); + if (!array_type) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Second argument of function {} must be an array, found {} instead.", + getName(), + arguments[1].type->getName()); + + const auto * merge_lambda = checkAndGetDataType(arguments[0].type.get()); + if (!merge_lambda) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument of function {} must be a lambda function, found {} instead.", + getName(), + arguments[0].type->getName()); + + DataTypePtr result_type; + if (arguments.size() == 3) + { + result_type = arguments[2].type; + } + else + { + const auto * finish_lambda = checkAndGetDataType(arguments[3].type.get()); + if (!finish_lambda) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Fourth argument of function {} must be a lambda function, found {} instead.", + getName(), + arguments[3].type->getName()); + result_type = finish_lambda->getReturnType(); + } + + if (arguments[1].type->isNullable() && !result_type->isNullable()) + result_type = makeNullable(result_type); + + return result_type; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const auto * merge_lambda = checkAndGetColumn(arguments[0].column.get()); + if (!merge_lambda) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "First argument of function {} must be a lambda function column, found {} instead.", + getName(), + arguments[0].column->getName()); + + ColumnPtr array_column = arguments[1].column->convertToFullColumnIfConst(); + DataTypePtr array_type = arguments[1].type; + const NullMap * array_null_map = nullptr; + if (const auto * nullable_array_column = checkAndGetColumn(array_column.get())) + { + array_column = nullable_array_column->getNestedColumnPtr(); + array_type = removeNullable(array_type); + array_null_map = &nullable_array_column->getNullMapData(); + } + + const auto * array_col = checkAndGetColumn(array_column.get()); + if (!array_col) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Second argument of function {} must be an array column, found {} instead.", + getName(), + array_column->getName()); + + ColumnPtr init_column = arguments[2].column->convertToFullColumnIfConst(); + DataTypePtr init_type = arguments[2].type; + + const auto * merge_lambda_type = checkAndGetDataType(arguments[0].type.get()); + if (!merge_lambda_type || merge_lambda_type->getArgumentTypes().size() != 2) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument of function {} must be a lambda function with 2 arguments, found {} instead.", + getName(), + arguments[0].type->getName()); + + const auto & merge_lambda_args = merge_lambda_type->getArgumentTypes(); + DataTypePtr acc_type = merge_lambda_args[0]; + DataTypePtr element_type = merge_lambda_args[1]; + + if (!init_type->equals(*acc_type)) + { + auto init_arg = ColumnWithTypeAndName(init_column, init_type, "acc"); + init_column = castColumn(init_arg, acc_type); + init_type = acc_type; + } + + const ColumnFunction * finish_lambda = nullptr; + if (arguments.size() == 4) + { + finish_lambda = checkAndGetColumn(arguments[3].column.get()); + if (!finish_lambda) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Fourth argument of function {} must be a lambda function column, found {} instead.", + getName(), + arguments[3].column->getName()); + } + + const auto & offsets = array_col->getOffsets(); + const auto & nested_data = array_col->getData(); + auto nested_type = assert_cast(*removeNullable(array_type)).getNestedType(); + const bool needs_element_cast = !nested_type->equals(*element_type); + + auto to_const_column = [](const MutableColumnPtr & column) -> ColumnPtr { + const IColumn & column_ref = *column; + return column_ref.getPtr(); + }; + auto make_single_value_column = [&](const DataTypePtr & type, const Field & value) { + auto col = type->createColumn(); + col->insert(value); + return to_const_column(col); + }; + + auto result_column = result_type->createColumn(); + result_column->reserve(input_rows_count); + + Field acc_field; + size_t previous_offset = 0; + for (size_t row = 0; row < input_rows_count; ++row) + { + if (array_null_map && (*array_null_map)[row]) + { + result_column->insertDefault(); + continue; + } + + init_column->get(row, acc_field); + size_t end_offset = offsets[row]; + for (size_t i = previous_offset; i < end_offset; ++i) + { + auto acc_col = make_single_value_column(init_type, acc_field); + auto element_col_mut = nested_data.cloneEmpty(); + element_col_mut->insertFrom(nested_data, i); + auto element_col = to_const_column(element_col_mut); + if (needs_element_cast) + { + auto element_arg = ColumnWithTypeAndName(element_col, nested_type, "element"); + element_col = castColumn(element_arg, element_type); + } + + auto lambda_clone = merge_lambda->cloneResized(1); + auto * lambda_col = typeid_cast(lambda_clone.get()); + lambda_col->appendArguments( + {ColumnWithTypeAndName(std::move(acc_col), init_type, "acc"), + ColumnWithTypeAndName(std::move(element_col), element_type, "element")}); + auto merged_col = lambda_col->reduce(); + merged_col.column->get(0, acc_field); + } + + if (finish_lambda) + { + const auto * finish_lambda_type = checkAndGetDataType(arguments[3].type.get()); + if (!finish_lambda_type || finish_lambda_type->getArgumentTypes().size() != 1) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Fourth argument of function {} must be a lambda function with 1 argument, found {} instead.", + getName(), + arguments[3].type->getName()); + + auto finish_arg_type = finish_lambda_type->getArgumentTypes().front(); + auto acc_col = make_single_value_column(init_type, acc_field); + if (!init_type->equals(*finish_arg_type)) + acc_col = castColumn(ColumnWithTypeAndName(acc_col, init_type, "acc"), finish_arg_type); + auto lambda_clone = finish_lambda->cloneResized(1); + auto * lambda_col = typeid_cast(lambda_clone.get()); + lambda_col->appendArguments({ColumnWithTypeAndName(std::move(acc_col), finish_arg_type, "acc")}); + auto finished_col = lambda_col->reduce(); + Field finished_field; + finished_col.column->get(0, finished_field); + result_column->insert(finished_field); + } + else + { + result_column->insert(acc_field); + } + + previous_offset = end_offset; + } + + return result_column; + } + + String getName() const override { return name; } +}; + +REGISTER_FUNCTION(SparkArrayFold) +{ + factory.registerFunction(); +} +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Functions/FunctionGetDateData.h b/cpp-ch/local-engine/Functions/FunctionGetDateData.h index f1523c4ce89a..9e455698d196 100644 --- a/cpp-ch/local-engine/Functions/FunctionGetDateData.h +++ b/cpp-ch/local-engine/Functions/FunctionGetDateData.h @@ -63,14 +63,14 @@ class FunctionGetDateData : public DB::IFunction for (size_t i = 0; i < size; ++i) { auto str = src_col->getDataAt(i); - if (str.size < 4) + if (str.size() < 4) { null_container[i] = true; continue; } else { - DB::ReadBufferFromMemory buf(str.data, str.size); + DB::ReadBufferFromMemory buf(str.data(), str.size()); while(!buf.eof() && *buf.position() == ' ') { buf.position() ++; diff --git a/cpp-ch/local-engine/Functions/FunctionsBloomFilter.h b/cpp-ch/local-engine/Functions/FunctionsBloomFilter.h index 6a6cdf27e781..de352e82a0bb 100644 --- a/cpp-ch/local-engine/Functions/FunctionsBloomFilter.h +++ b/cpp-ch/local-engine/Functions/FunctionsBloomFilter.h @@ -134,7 +134,7 @@ class FunctionBloomFilterContains : public DB::IFunction = *reinterpret_cast(bloom_filter_state); if (second_arg_const) { - vec_to[0] = bloom_filter_data_0.bloom_filter.find(typeid_cast(*column_ptr).getDataAt(0).data, sizeof(T)); + vec_to[0] = bloom_filter_data_0.bloom_filter.find(typeid_cast(*column_ptr).getDataAt(0).data(), sizeof(T)); // copy to all rows, better use constant column std::memcpy(&vec_to[1], &vec_to[0], (input_rows_count - 1) * sizeof(UInt8)); @@ -177,14 +177,14 @@ class FunctionBloomFilterContains : public DB::IFunction { if (isColumnConst(*first_column_ptr)) first_column_ptr = &typeid_cast(*first_column_ptr).getDataColumn(); - StringRef sr = typeid_cast(*first_column_ptr).getDataAt(0); + std::string_view sr = typeid_cast(*first_column_ptr).getDataAt(0); size_t size_of_state = agg_func->sizeOfData(); allocated_bytes_for_bloom_filter_state = new char[size_of_state]; agg_func->create(allocated_bytes_for_bloom_filter_state); if (!sr.empty()) { - DB::ReadBufferFromMemory read_buffer(sr.data, sr.size); + DB::ReadBufferFromMemory read_buffer(sr.data(), sr.size()); agg_func->deserialize((allocated_bytes_for_bloom_filter_state), read_buffer); } } diff --git a/cpp-ch/local-engine/Functions/LocalDigitsToAsciiDigitForDate.cpp b/cpp-ch/local-engine/Functions/LocalDigitsToAsciiDigitForDate.cpp index 1e50e02bff53..4efa96143537 100644 --- a/cpp-ch/local-engine/Functions/LocalDigitsToAsciiDigitForDate.cpp +++ b/cpp-ch/local-engine/Functions/LocalDigitsToAsciiDigitForDate.cpp @@ -169,7 +169,7 @@ class LocalDigitsToAsciiDigitForDateFunction : public DB::IFunction LOG_DEBUG( getLogger("LocalDigitsToAsciiDigitForDateFunction"), "Converted local digit string {} to ascii digit string: {}", - col_str->getDataAt(row_index).toString(), + String(col_str->getDataAt(row_index).data(), col_str->getDataAt(row_index).size()), converted); res_col->insertData(converted.c_str(), converted.size()); } @@ -231,18 +231,18 @@ class LocalDigitsToAsciiDigitForDateFunction : public DB::IFunction #endif } - bool convertLocalDigitIfNeeded(StringRef str, std::string & result) const + bool convertLocalDigitIfNeeded(std::string_view str, std::string & result) const { - if (!str.size) + if (!str.size()) return false; - if (!hasNonAsciiSimd(str.data, str.size)) + if (!hasNonAsciiSimd(str.data(), str.size())) return false; result.clear(); - result.reserve(str.size); + result.reserve(str.size()); bool has_local_digit = false; - for (size_t i = 0; i < str.size;) + for (size_t i = 0; i < str.size();) { - unsigned char c = str.data[i]; + unsigned char c = str.data()[i]; char32_t cp = 0; if ((c & 0x80) == 0) // 1-byte { @@ -252,7 +252,7 @@ class LocalDigitsToAsciiDigitForDateFunction : public DB::IFunction } else if ((c & 0xE0) == 0xC0) // 2-byte { - unsigned char b1 = str.data[i + 1]; + unsigned char b1 = str.data()[i + 1]; if (c == 0xD9 && b1 >= 0xA0 && b1 <= 0xA9) // Arabic-Indic { result.push_back(static_cast('0' + (b1 - 0xA0))); @@ -284,8 +284,8 @@ class LocalDigitsToAsciiDigitForDateFunction : public DB::IFunction } else if ((c & 0xF0) == 0xE0) // 3-byte { - unsigned char b1 = str.data[i + 1]; - unsigned char b2 = str.data[i + 2]; + unsigned char b1 = str.data()[i + 1]; + unsigned char b2 = str.data()[i + 2]; if (c == 0xE0) { if ((b1 == 0xA5 && b2 >= 0xA6 && b2 <= 0xAF) || // Devanagari @@ -329,9 +329,9 @@ class LocalDigitsToAsciiDigitForDateFunction : public DB::IFunction } else if ((c & 0xF8) == 0xF0) // 4-byte { - unsigned char b1 = str.data[i + 1]; - unsigned char b2 = str.data[i + 2]; - unsigned char b3 = str.data[i + 3]; + unsigned char b1 = str.data()[i + 1]; + unsigned char b2 = str.data()[i + 2]; + unsigned char b3 = str.data()[i + 3]; cp = ((c & 0x07) << 18) | ((b1 & 0x3F) << 12) | ((b2 & 0x3F) << 6) | (b3 & 0x3F); auto local_digit = toAsciiDigit(cp); if (local_digit) diff --git a/cpp-ch/local-engine/Functions/SparkFunctionArrayDistinct.cpp b/cpp-ch/local-engine/Functions/SparkFunctionArrayDistinct.cpp index 0779346b6e04..448172200cdc 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionArrayDistinct.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionArrayDistinct.cpp @@ -223,7 +223,7 @@ bool FunctionArrayDistinctSpark::executeString( if (!src_data_concrete) return false; - using Set = ClearableHashSetWithStackMemory; const PaddedPODArray * src_null_map = nullptr; @@ -253,12 +253,12 @@ bool FunctionArrayDistinctSpark::executeString( continue; } - StringRef str_ref = src_data_concrete->getDataAt(j); + std::string_view str_ref = src_data_concrete->getDataAt(j); if (!set.find(str_ref)) { set.insert(str_ref); - res_data_col.insertData(str_ref.data, str_ref.size); + res_data_col.insertData(str_ref.data(), str_ref.size()); } } diff --git a/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp b/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp index 89b07f263f07..3b6daa56d367 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp @@ -23,7 +23,6 @@ #include #include #include -#include using namespace DB; @@ -107,8 +106,8 @@ class SparkFunctionArrayJoin : public IFunction for (size_t i = 0; i < array_col->size(); ++i) { String res; - const StringRef delim = const_delim_col ? delim_col->getDataAt(0) : delim_col->getDataAt(i); - StringRef null_replacement = StringRef(nullptr, 0); + const std::string_view delim = const_delim_col ? delim_col->getDataAt(0) : delim_col->getDataAt(i); + std::string_view null_replacement = std::string_view(nullptr, 0); if (null_replacement_col) { null_replacement = const_null_replacement_col ? null_replacement_col->getDataAt(0) : null_replacement_col->getDataAt(i); @@ -120,22 +119,24 @@ class SparkFunctionArrayJoin : public IFunction { if (array_nested_col && array_nested_col->isNullAt(j + array_pos)) { - if (null_replacement.data) + if (null_replacement.data()) { - res += null_replacement.toString(); + res += String(null_replacement.data(), null_replacement.size()); if (j != array_size - 1) - res += delim.toString(); + res += String(delim.data(), delim.size()); } else if (j == array_size - 1) res = res.substr(0, last_not_null_pos); } else { - const StringRef s(&string_data[data_pos], string_offsets[j + array_pos] - data_pos); - res += s.toString(); + // TODO: rebase-25.12, is it correct to reinterpret_cast + const std::string_view s( + reinterpret_cast(&string_data[data_pos]), string_offsets[j + array_pos] - data_pos); + res += String(s.data(), s.size()); last_not_null_pos = res.size(); if (j != array_size - 1) - res += delim.toString(); + res += String(delim.data(), delim.size()); } data_pos = string_offsets[j + array_pos]; } diff --git a/cpp-ch/local-engine/Functions/SparkFunctionConv.cpp b/cpp-ch/local-engine/Functions/SparkFunctionConv.cpp index fab3e6612600..4d8505b36b95 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionConv.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionConv.cpp @@ -195,7 +195,8 @@ DB::ColumnPtr SparkFunctionConv::executeImpl( int err = 0; for (size_t i = 0; i < input_rows_count; ++i) { - auto value_str = arguments[0].column->getDataAt(i).toString(); + auto str_data = arguments[0].column->getDataAt(i); + auto value_str = String(str_data.data(), str_data.size()); if (from_base < 0) dec = my_strntoull_8bit(value_str.data(), value_str.length(), -from_base, &endptr, &err); else diff --git a/cpp-ch/local-engine/Functions/SparkFunctionGetJsonObject.cpp b/cpp-ch/local-engine/Functions/SparkFunctionGetJsonObject.cpp index 869221ce7724..0b097da5c96d 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionGetJsonObject.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionGetJsonObject.cpp @@ -21,14 +21,14 @@ namespace local_engine { -std::pair JSONPathNormalizer::prevToken(DB::IParser::Pos & iter, size_t n) +std::pair JSONPathNormalizer::prevToken(DB::IParser::Pos & iter, size_t n) { size_t i = 0; for (; i < n && iter->type != DB::TokenType::DollarSign; ++i) { --iter; } - std::pair res = {iter->type, StringRef(iter->begin, iter->end - iter->begin)}; + std::pair res = {iter->type, std::string_view(iter->begin, iter->end - iter->begin)}; for (; i > 0; --i) { ++iter; @@ -36,14 +36,14 @@ std::pair JSONPathNormalizer::prevToken(DB::IParser::P return res; } -std::pair JSONPathNormalizer::nextToken(DB::IParser::Pos & iter, size_t n) +std::pair JSONPathNormalizer::nextToken(DB::IParser::Pos & iter, size_t n) { size_t i = 0; for (; i < n && iter->type != DB::TokenType::EndOfStream; ++i) { ++iter; } - std::pair res = {iter->type, StringRef(iter->begin, iter->end - iter->begin)}; + std::pair res = {iter->type, std::string_view(iter->begin, iter->end - iter->begin)}; for (; i > 0; --i) { --iter; @@ -74,16 +74,16 @@ void JSONPathNormalizer::normalizeOnNumber(DB::IParser::Pos & iter, String & res { auto [_, prev_iter_str] = prevToken(iter); // may contains spaces - if (prev_iter_str.data + prev_iter_str.size != iter->begin) + if (prev_iter_str.data() + prev_iter_str.size() != iter->begin) { - res += String(prev_iter_str.data + prev_iter_str.size, iter->begin); + res += String(prev_iter_str.data() + prev_iter_str.size(), iter->begin); } res += String(iter->begin, iter->end); ++iter; token_type = iter->type; } auto [_, prev_iter_str] = prevToken(iter); - res += String(prev_iter_str.data + prev_iter_str.size, iter->begin); + res += String(prev_iter_str.data() + prev_iter_str.size(), iter->begin); res += "\""; } else @@ -110,9 +110,9 @@ void JSONPathNormalizer::normalizeOnBareWord(DB::IParser::Pos & iter, String & r { auto [_, prev_iter_str] = prevToken(iter); // may contains spaces - if (prev_iter_str.data + prev_iter_str.size != iter->begin) + if (prev_iter_str.data() + prev_iter_str.size() != iter->begin) { - res += String(prev_iter_str.data + prev_iter_str.size, iter->begin); + res += String(prev_iter_str.data() + prev_iter_str.size(), iter->begin); } } res += String(iter->begin, iter->end); @@ -121,7 +121,7 @@ void JSONPathNormalizer::normalizeOnBareWord(DB::IParser::Pos & iter, String & r token_type = iter->type; } auto [_, prev_iter_str] = prevToken(iter); - res += String(prev_iter_str.data + prev_iter_str.size, iter->begin); + res += String(prev_iter_str.data() + prev_iter_str.size(), iter->begin); res += "\""; } } diff --git a/cpp-ch/local-engine/Functions/SparkFunctionGetJsonObject.h b/cpp-ch/local-engine/Functions/SparkFunctionGetJsonObject.h index 7183e4e8b5df..f3ccd8fec17a 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionGetJsonObject.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionGetJsonObject.h @@ -570,9 +570,9 @@ class JSONPathNormalizer } private: - static std::pair prevToken(DB::IParser::Pos & iter, size_t n = 1); + static std::pair prevToken(DB::IParser::Pos & iter, size_t n = 1); - static std::pair nextToken(DB::IParser::Pos & iter, size_t n = 1); + static std::pair nextToken(DB::IParser::Pos & iter, size_t n = 1); static bool isSubPathBegin(DB::IParser::Pos & iter); @@ -602,7 +602,8 @@ class FlattenJSONStringOnRequiredFunction : public DB::IFunction String json_fields; if (const auto * json_fields_col = typeid_cast(arguments[1].column.get())) { - json_fields = json_fields_col->getDataAt(0).toString(); + const auto str_json_fields = json_fields_col->getDataAt(0); + json_fields = std::string(str_json_fields.data(), str_json_fields.size()); } else { @@ -692,7 +693,8 @@ class FlattenJSONStringOnRequiredFunction : public DB::IFunction const auto & first_column = arguments[0]; if (const auto * required_fields_col = typeid_cast(arguments[1].column.get())) { - std::string json_fields = required_fields_col->getDataAt(0).toString(); + const auto str_json_fields = required_fields_col->getDataAt(0); + std::string json_fields = std::string(str_json_fields.data(), str_json_fields.size()); Poco::StringTokenizer tokenizer(json_fields, "|"); bool path_parsed = true; for (const auto & field : tokenizer) diff --git a/cpp-ch/local-engine/Functions/SparkFunctionHashingExtended.h b/cpp-ch/local-engine/Functions/SparkFunctionHashingExtended.h index a1acccc1ca60..ac3177d44f1c 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionHashingExtended.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionHashingExtended.h @@ -347,10 +347,10 @@ class SparkFunctionAnyHash : public DB::IFunction { if (!null_map || !(*null_map)[0]) [[likely]] { - StringRef ref = col_from_fixed->getDataAt(0); + std::string_view ref = col_from_fixed->getDataAt(0); for (size_t i = 0; i < size; ++i) - vec_to[i] = applyUnsafeBytes(ref.data, ref.size, vec_to[i]); + vec_to[i] = applyUnsafeBytes(ref.data(), ref.size(), vec_to[i]); } } } @@ -382,10 +382,10 @@ class SparkFunctionAnyHash : public DB::IFunction { if (!null_map || !(*null_map)[0]) [[likely]] { - StringRef ref = col_from->getDataAt(0); + std::string_view ref = col_from->getDataAt(0); for (size_t i = 0; i < size; ++i) - vec_to[i] = applyUnsafeBytes(ref.data, ref.size, vec_to[i]); + vec_to[i] = applyUnsafeBytes(ref.data(), ref.size(), vec_to[i]); } } } diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMonthsBetween.cpp b/cpp-ch/local-engine/Functions/SparkFunctionMonthsBetween.cpp index 125bf3bc2b84..72b197ae88cd 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMonthsBetween.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionMonthsBetween.cpp @@ -115,9 +115,9 @@ DB::ColumnPtr SparkFunctionMonthsBetween::executeImpl( size_t rows = input_rows_count; auto res = result_type->createColumn(); res->reserve(rows); - std::string timezone_str = ""; + std::string_view timezone_str = ""; if (arguments.size() == 4 && rows) // We know that timezone is constant - timezone_str = arguments[3].column->getDataAt(0).toString(); + timezone_str = arguments[3].column->getDataAt(0); auto & timezone = DateLUT::instance(timezone_str); for (size_t i = 0; i < rows; ++i) diff --git a/cpp-ch/local-engine/Functions/SparkFunctionReinterpretAsString.cpp b/cpp-ch/local-engine/Functions/SparkFunctionReinterpretAsString.cpp index 08ff46045c22..52acea1c1ace 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionReinterpretAsString.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionReinterpretAsString.cpp @@ -84,7 +84,8 @@ namespace { /// Transform little-endian in input to big-endian in output /// NOTE: We don't need do the transform for string type - String data = src.getDataAt(i).toString(); + auto str_data = src.getDataAt(i); + String data = String(str_data.data(), str_data.size()); if (!is_string_type) std::reverse(data.begin(), data.end()); diff --git a/cpp-ch/local-engine/Functions/SparkFunctionStrToMap.cpp b/cpp-ch/local-engine/Functions/SparkFunctionStrToMap.cpp index 84940af869c5..8936ea0d04b0 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionStrToMap.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionStrToMap.cpp @@ -109,7 +109,7 @@ struct RegularSplitter RegularSplitter(const String & delimiter_) : delimiter(delimiter_) { if (!delimiter.empty()) - re = std::make_shared(DB::Regexps::createRegexp(delimiter)); + re = std::make_shared(DB::Regexps::createRegexp(delimiter)); } void reset(Pos str_begin_, Pos str_end_) @@ -160,7 +160,7 @@ struct RegularSplitter private: String delimiter; DB::Regexps::RegexpPtr re; - OptimizedRegularExpression::MatchVec matches; + DB::OptimizedRegularExpression::MatchVec matches; Pos str_begin; Pos str_end; Pos str_cursor; @@ -372,7 +372,7 @@ class SparkFunctionStrToMapOverloadResolver : public DB::IFunctionOverloadResolv String pattern = col->getValue(); if (pattern.empty()) return false; - OptimizedRegularExpression re = DB::Regexps::createRegexp(pattern); + DB::OptimizedRegularExpression re = DB::Regexps::createRegexp(pattern); std::string required_substring; bool is_trivial; diff --git a/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h b/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h index aae0a359f59e..07fc53db490a 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h @@ -164,8 +164,8 @@ class SparkFunctionConvertToDateTime : public DB::IFunction const DateLUTImpl * utc_time_zone = &DateLUT::instance("UTC"); for (size_t i = 0; i < src->size(); ++i) { - const StringRef data = src->getDataAt(i); - DB::ReadBufferFromMemory buf(data.data, data.size); + const std::string_view data = src->getDataAt(i); + DB::ReadBufferFromMemory buf(data.data(), data.size()); while(!buf.eof() && *buf.position() == ' ') { buf.position() ++; diff --git a/cpp-ch/local-engine/Functions/SparkFunctionTrim.cpp b/cpp-ch/local-engine/Functions/SparkFunctionTrim.cpp index 348bccba5e54..940e9bcb0ee6 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionTrim.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionTrim.cpp @@ -138,8 +138,8 @@ namespace res_data.reserve_exact(src_const_str.size() * input_rows_count); for (size_t row = 0; row < input_rows_count; ++row) { - StringRef trim_str_ref = trim_col->getDataAt(row); - std::unique_ptr> trim_set = buildTrimSet(trim_str_ref.data, trim_str_ref.size); + std::string_view trim_str_ref = trim_col->getDataAt(row); + std::unique_ptr> trim_set = buildTrimSet(trim_str_ref.data(), trim_str_ref.size()); executeRow(src_const_str.c_str(), src_const_str.size(), res_data, res_offsets, row, *trim_set); } return std::move(res_col); @@ -152,8 +152,8 @@ namespace std::unique_ptr> trim_set = buildTrimSet(trim_const_str.c_str(), trim_const_str.size()); for (size_t row = 0; row < input_rows_count; ++row) { - StringRef src_str_ref = src_col->getDataAt(row); - executeRow(src_str_ref.data, src_str_ref.size, res_data, res_offsets, row, *trim_set); + std::string_view src_str_ref = src_col->getDataAt(row); + executeRow(src_str_ref.data(), src_str_ref.size(), res_data, res_offsets, row, *trim_set); } return std::move(res_col); } @@ -162,10 +162,10 @@ namespace res_data.reserve(src_col->getChars().size()); for (size_t row = 0; row < input_rows_count; ++row) { - StringRef src_str_ref = src_col->getDataAt(row); - StringRef trim_str_ref = trim_col->getDataAt(row); - std::unique_ptr> trim_set = buildTrimSet(trim_str_ref.data, trim_str_ref.size); - executeRow(src_str_ref.data, src_str_ref.size, res_data, res_offsets, row, *trim_set); + std::string_view src_str_ref = src_col->getDataAt(row); + std::string_view trim_str_ref = trim_col->getDataAt(row); + std::unique_ptr> trim_set = buildTrimSet(trim_str_ref.data(), trim_str_ref.size()); + executeRow(src_str_ref.data(), src_str_ref.size(), res_data, res_offsets, row, *trim_set); } return std::move(res_col); } diff --git a/cpp-ch/local-engine/Functions/SparkParseURL.cpp b/cpp-ch/local-engine/Functions/SparkParseURL.cpp index 97fa819acaa8..246dd3f9c6cc 100644 --- a/cpp-ch/local-engine/Functions/SparkParseURL.cpp +++ b/cpp-ch/local-engine/Functions/SparkParseURL.cpp @@ -242,7 +242,8 @@ struct SparkExtractURLOneQuery { try { - const String s = col.getDataAt(i).toString(); + const auto str_data = col.getDataAt(i); + const String s = String(str_data.data(), str_data.size()); Poco::URI uri(s, false); String protocol_prefix = uri.getScheme() + "://"; diff --git a/cpp-ch/local-engine/Functions/SparkPartitionEscape.cpp b/cpp-ch/local-engine/Functions/SparkPartitionEscape.cpp index 522f9ddee2b7..bba1cbfeb188 100644 --- a/cpp-ch/local-engine/Functions/SparkPartitionEscape.cpp +++ b/cpp-ch/local-engine/Functions/SparkPartitionEscape.cpp @@ -96,7 +96,8 @@ DB::ColumnPtr SparkPartitionEscape::executeImpl( for (size_t i = 0; i < input_rows_count; ++i) { - auto escaped_name = escapePathName(arguments[0].column->getDataAt(i).toString()); + const auto escaped_name_data = arguments[0].column->getDataAt(i); + auto escaped_name = escapePathName(std::string(escaped_name_data.data(), escaped_name_data.size())); result->insertData(escaped_name.c_str(), escaped_name.size()); } return result; diff --git a/cpp-ch/local-engine/IO/SplittableBzip2ReadBuffer.h b/cpp-ch/local-engine/IO/SplittableBzip2ReadBuffer.h index 375a6c885812..d682c0a9ee4e 100644 --- a/cpp-ch/local-engine/IO/SplittableBzip2ReadBuffer.h +++ b/cpp-ch/local-engine/IO/SplittableBzip2ReadBuffer.h @@ -22,7 +22,6 @@ #if USE_BZIP2 #include #include -#include #include namespace DB diff --git a/cpp-ch/local-engine/Operator/FillingDeltaInternalRowDeletedStep.cpp b/cpp-ch/local-engine/Operator/FillingDeltaInternalRowDeletedStep.cpp index ca8f0f1acc17..2c92f5aa1fa2 100644 --- a/cpp-ch/local-engine/Operator/FillingDeltaInternalRowDeletedStep.cpp +++ b/cpp-ch/local-engine/Operator/FillingDeltaInternalRowDeletedStep.cpp @@ -109,7 +109,8 @@ void FillingDeltaInternalRowDeletedTransform::transform(DB::Chunk & chunk) for (size_t i = 0; i < num_rows; ++i) { - std::string part_name = part_path_key_column_data.getDataAt(i).toString(); + const std::string_view str_data = part_path_key_column_data.getDataAt(i); + std::string part_name = std::string(str_data.data(), str_data.size()); if (dv_map.contains(part_name)) { vec[i] = dv_map.at(part_name)->rb_contains(row_index_column_data.get64(i)); diff --git a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp index 4be7a14065c8..92458c561f99 100644 --- a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp +++ b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp @@ -348,7 +348,7 @@ size_t GraceAggregatingTransform::flushBucket(size_t bucket_index) if (!file_stream.original_blocks.empty()) { if (!file_stream.original_file_stream) - file_stream.original_file_stream = DB::TemporaryBlockStreamHolder(header, tmp_data_disk.get()); + file_stream.original_file_stream = DB::TemporaryBlockStreamHolder(header, tmp_data_disk); flush_bytes += flushBlocksInfoDisk(file_stream.original_file_stream, file_stream.original_blocks); } if (!file_stream.intermediate_blocks.empty()) @@ -356,7 +356,7 @@ size_t GraceAggregatingTransform::flushBucket(size_t bucket_index) if (!file_stream.intermediate_file_stream) { auto intermediate_header = toShared(params->aggregator.getHeader(false)); - file_stream.intermediate_file_stream = DB::TemporaryBlockStreamHolder(intermediate_header, tmp_data_disk.get()); + file_stream.intermediate_file_stream = DB::TemporaryBlockStreamHolder(intermediate_header, tmp_data_disk); } flush_bytes += flushBlocksInfoDisk(file_stream.intermediate_file_stream, file_stream.intermediate_blocks); } diff --git a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp index 696f5352f200..7e83356bf84c 100644 --- a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp @@ -196,7 +196,7 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::convertNodeTypeIfNeeded( bool need_convert_type = !TypeParser::isTypeMatched(output_type, func_node->result_type, !with_nullability); if (need_convert_type) { - func_node = ActionsDAGUtil::convertNodeType(actions_dag, func_node, TypeParser::parseType(output_type), func_node->result_name); + func_node = ActionsDAGUtil::convertNodeType(actions_dag, func_node, TypeParser::parseType(output_type), getContext(), func_node->result_name); actions_dag.addOrReplaceInOutputs(*func_node); } @@ -225,8 +225,7 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::convertNanToNullIfNeed( /// result is nullable. /// if result is NaN, convert it to NULL. auto is_nan_func_node = toFunctionNode(actions_dag, "isNaN", getUniqueName("isNaN"), {func_node}); - auto nullable_col = func_node->result_type->createColumn(); - nullable_col->insertDefault(); + auto nullable_col = func_node->result_type->createColumnConst(1, func_node->result_type->getDefault()); const auto * null_node = &actions_dag.addColumn(DB::ColumnWithTypeAndName(std::move(nullable_col), func_node->result_type, getUniqueName("null"))); DB::ActionsDAG::NodeRawConstPtrs convert_nan_func_args = {is_nan_func_node, null_node, func_node}; diff --git a/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp b/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp index 4ada9bfc5a45..0c4e1ff68c7a 100644 --- a/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp +++ b/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include @@ -170,8 +169,8 @@ static void writeVariableLengthNonNullableValue( for (size_t i = 0; i < num_rows; i++) { size_t row_idx = masks == nullptr ? i : masks->at(i); - StringRef str = col.column->getDataAt(row_idx); - int64_t offset_and_size = writer.writeUnalignedBytes(i, str.data, str.size, 0); + std::string_view str = col.column->getDataAt(row_idx); + int64_t offset_and_size = writer.writeUnalignedBytes(i, str.data(), str.size(), 0); memcpy(buffer_address + offsets[i] + field_offset, &offset_and_size, 8); } } @@ -181,8 +180,8 @@ static void writeVariableLengthNonNullableValue( for (size_t i = 0; i < num_rows; i++) { size_t row_idx = masks == nullptr ? i : masks->at(i); - StringRef str_view = col.column->getDataAt(row_idx); - String buf(str_view.data, str_view.size); + std::string_view str_view = col.column->getDataAt(row_idx); + String buf(str_view.data(), str_view.size()); BackingDataLengthCalculator::swapDecimalEndianBytes(buf); int64_t offset_and_size = writer.writeUnalignedBytes(i, buf.data(), buf.size(), 0); memcpy(buffer_address + offsets[i] + field_offset, &offset_and_size, 8); @@ -228,16 +227,16 @@ static void writeVariableLengthNullableValue( bitSet(buffer_address + offsets[i], col_index); else if (!big_endian) { - StringRef str = nested_column.getDataAt(row_idx); - int64_t offset_and_size = writer.writeUnalignedBytes(i, str.data, str.size, 0); + std::string_view str = nested_column.getDataAt(row_idx); + int64_t offset_and_size = writer.writeUnalignedBytes(i, str.data(), str.size(), 0); memcpy(buffer_address + offsets[i] + field_offset, &offset_and_size, 8); } else { Field field; nested_column.get(row_idx, field); - StringRef str_view = nested_column.getDataAt(row_idx); - String buf(str_view.data, str_view.size); + std::string_view str_view = nested_column.getDataAt(row_idx); + String buf(str_view.data(), str_view.size()); BackingDataLengthCalculator::swapDecimalEndianBytes(buf); int64_t offset_and_size = writer.writeUnalignedBytes(i, buf.data(), buf.size(), 0); memcpy(buffer_address + offsets[i] + field_offset, &offset_and_size, 8); @@ -332,7 +331,7 @@ SparkRowInfo::SparkRowInfo( { size_t row_idx = masks == nullptr ? i : masks->at(i); if (!null_map[row_idx]) - lengths[i] += roundNumberOfBytesToNearestWord(nested_column.getDataAt(row_idx).size); + lengths[i] += roundNumberOfBytesToNearestWord(nested_column.getDataAt(row_idx).size()); } } else @@ -340,7 +339,7 @@ SparkRowInfo::SparkRowInfo( for (size_t i = 0; i < num_rows; ++i) { size_t row_idx = masks == nullptr ? i : masks->at(i); - lengths[i] += roundNumberOfBytesToNearestWord(column->getDataAt(row_idx).size); + lengths[i] += roundNumberOfBytesToNearestWord(column->getDataAt(row_idx).size()); } } } @@ -936,9 +935,9 @@ void FixedLengthDataWriter::write(const DB::Field & field, char * buffer) throw Exception(ErrorCodes::UNKNOWN_TYPE, "FixedLengthDataWriter doesn't support type {}", type_without_nullable->getName()); } -void FixedLengthDataWriter::unsafeWrite(const StringRef & str, char * buffer) +void FixedLengthDataWriter::unsafeWrite(const std::string_view & str, char * buffer) { - memcpy(buffer, str.data, str.size); + memcpy(buffer, str.data(), str.size()); } void FixedLengthDataWriter::unsafeWrite(const char * __restrict src, char * __restrict buffer) diff --git a/cpp-ch/local-engine/Parser/CHColumnToSparkRow.h b/cpp-ch/local-engine/Parser/CHColumnToSparkRow.h index c85f0710de57..d30a5d767bcb 100644 --- a/cpp-ch/local-engine/Parser/CHColumnToSparkRow.h +++ b/cpp-ch/local-engine/Parser/CHColumnToSparkRow.h @@ -23,8 +23,6 @@ #include #include -struct StringRef; - namespace local_engine { int64_t calculateBitSetWidthInBytes(int64_t num_fields); @@ -187,7 +185,7 @@ class FixedLengthDataWriter /// Copy memory chunk of Fixed length typed CH Column directory to buffer for performance. /// It is unsafe unless you know what you are doing. - virtual void unsafeWrite(const StringRef & str, char * buffer); + virtual void unsafeWrite(const std::string_view & str, char * buffer); /// Copy memory chunk of in fixed length typed Field directory to buffer for performance. /// It is unsafe unless you know what you are doing. diff --git a/cpp-ch/local-engine/Parser/ExpressionParser.cpp b/cpp-ch/local-engine/Parser/ExpressionParser.cpp index 12392a9dfe13..a6fe76c2f16c 100644 --- a/cpp-ch/local-engine/Parser/ExpressionParser.cpp +++ b/cpp-ch/local-engine/Parser/ExpressionParser.cpp @@ -262,7 +262,7 @@ bool ExpressionParser::reuseCSE() const ExpressionParser::NodeRawConstPtr ExpressionParser::addConstColumn(DB::ActionsDAG & actions_dag, const DB::DataTypePtr & type, const DB::Field & field) const { - String name = toString(field).substr(0, 10); + String name = DB::fieldToString(field).substr(0, 10); name = getUniqueName(name); const auto * res_node = &actions_dag.addColumn(DB::ColumnWithTypeAndName(type->createColumnConst(1, field), type, name)); if (reuseCSE()) @@ -793,7 +793,7 @@ ExpressionParser::parseArrayJoin(const substrait::Expression_ScalarFunction & fu /// pos = cast(arrayJoin(arg_not_null).1, "Int32") const auto * pos_node = add_tuple_element(array_join_node, 1); - pos_node = ActionsDAGUtil::convertNodeType(actions_dag, pos_node, INT()); + pos_node = ActionsDAGUtil::convertNodeType(actions_dag, pos_node, INT(), context->queryContext()); /// if is_map is false, output col = arrayJoin(arg_not_null).2 /// if is_map is true, output (key, value) = arrayJoin(arg_not_null).2 diff --git a/cpp-ch/local-engine/Parser/FunctionParser.cpp b/cpp-ch/local-engine/Parser/FunctionParser.cpp index 3d9da46ad1e2..7348f5d286ba 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/FunctionParser.cpp @@ -131,6 +131,7 @@ const ActionsDAG::Node * FunctionParser::convertNodeTypeIfNeeded( // as stated in isTypeMatched, currently we don't change nullability of the result type func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, result_type) : local_engine::removeNullable(result_type), + getContext(), func_node->result_name, CastType::accurateOrNull); } @@ -139,7 +140,7 @@ const ActionsDAG::Node * FunctionParser::convertNodeTypeIfNeeded( // as stated in isTypeMatched, currently we don't change nullability of the result type auto target_type = func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, result_type) : local_engine::removeNullable(result_type); - return ActionsDAGUtil::convertNodeType(actions_dag, func_node, target_type, func_node->result_name); + return ActionsDAGUtil::convertNodeType(actions_dag, func_node, target_type, getContext(), func_node->result_name); } } else diff --git a/cpp-ch/local-engine/Parser/RelMetric.cpp b/cpp-ch/local-engine/Parser/RelMetric.cpp index 98a9b284ec6b..4de98fdde426 100644 --- a/cpp-ch/local-engine/Parser/RelMetric.cpp +++ b/cpp-ch/local-engine/Parser/RelMetric.cpp @@ -143,7 +143,7 @@ void RelMetric::serialize(Writer & writer, bool) const writer.Key("name"); writer.String(step->getName().c_str()); writer.Key("description"); - writer.String(step->getStepDescription().c_str()); + writer.String(step->getStepDescription().data()); writer.Key("processors"); writer.StartArray(); for (const auto & processor : step->getProcessors()) diff --git a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp index 3856b71781f0..f81d8ec77b29 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp @@ -20,7 +20,6 @@ #include #include #include -#include #include #include #include @@ -253,7 +252,7 @@ void AggregateRelParser::buildAggregateDescriptions(AggregateDescriptions & desc { if (i != 0) result += ","; - result += toString(params[i]); + result += DB::fieldToString(params[i]); } result += ")"; } diff --git a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp index ce38f7c43a9f..716cd227348e 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp @@ -138,7 +138,7 @@ void CrossRelParser::renamePlanColumns(DB::QueryPlan & left, DB::QueryPlan & rig if (right_ori_header.size() > 0 && right_ori_header[0].name != BlockUtil::VIRTUAL_ROW_COUNT_COLUMN) { ActionsDAG right_project = ActionsDAG::makeConvertingActions( - right_ori_header, storage_join.getRightSampleBlock().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); + right_ori_header, storage_join.getRightSampleBlock().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position, getContext()); QueryPlanStepPtr project_step = std::make_unique(right.getCurrentHeader(), std::move(right_project)); project_step->setStepDescription("Rename Broadcast Table Name"); steps.emplace_back(project_step.get()); @@ -157,7 +157,7 @@ void CrossRelParser::renamePlanColumns(DB::QueryPlan & left, DB::QueryPlan & rig else new_left_cols.emplace_back(col.column, col.type, col.name); auto left_header = left.getCurrentHeader()->getColumnsWithTypeAndName(); - ActionsDAG left_project = ActionsDAG::makeConvertingActions(left_header, new_left_cols, ActionsDAG::MatchColumnsMode::Position); + ActionsDAG left_project = ActionsDAG::makeConvertingActions(left_header, new_left_cols, ActionsDAG::MatchColumnsMode::Position, getContext()); QueryPlanStepPtr project_step = std::make_unique(left.getCurrentHeader(), std::move(left_project)); project_step->setStepDescription("Rename Left Table Name for broadcast join"); @@ -322,7 +322,7 @@ void CrossRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left std::optional left_convert_actions; std::optional right_convert_actions; std::tie(left_convert_actions, right_convert_actions) = table_join.createConvertingActions( - left.getCurrentHeader()->getColumnsWithTypeAndName(), right.getCurrentHeader()->getColumnsWithTypeAndName()); + left.getCurrentHeader()->getColumnsWithTypeAndName(), right.getCurrentHeader()->getColumnsWithTypeAndName(), getContext()); if (right_convert_actions) { diff --git a/cpp-ch/local-engine/Parser/RelParsers/FilterRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/FilterRelParser.cpp index e3cae75a1acd..fc09b2dc5f7e 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/FilterRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/FilterRelParser.cpp @@ -66,7 +66,7 @@ FilterRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, // header maybe changed, need to rollback it if (!blocksHaveEqualStructure(input_header, *query_plan->getCurrentHeader())) { - steps.emplace_back(PlanUtil::adjustQueryPlanHeader(*query_plan, input_header, "Rollback filter header")); + steps.emplace_back(PlanUtil::adjustQueryPlanHeader(*query_plan, input_header, "Rollback filter header", parser_context->queryContext())); } // remove nullable diff --git a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp index 5aee65474108..2d49a8b453a7 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp @@ -239,7 +239,8 @@ DB::QueryPlanPtr AggregateGroupLimitRelParser::parse( auto convert_actions_dag = DB::ActionsDAG::makeConvertingActions( window_plan->getCurrentHeader()->getColumnsWithTypeAndName(), aggregation_plan->getCurrentHeader()->getColumnsWithTypeAndName(), - DB::ActionsDAG::MatchColumnsMode::Position); + DB::ActionsDAG::MatchColumnsMode::Position, + getContext()); auto convert_step = std::make_unique(window_plan->getCurrentHeader(), std::move(convert_actions_dag)); convert_step->setStepDescription("Rename rank column name"); window_plan->addStep(std::move(convert_step)); @@ -439,7 +440,8 @@ void AggregateGroupLimitRelParser::postProjectionForExplodingArrays(DB::QueryPla auto adjust_pos_actions_dag = DB::ActionsDAG::makeConvertingActions( flatten_tuple_output_header->getColumnsWithTypeAndName(), output_header.getColumnsWithTypeAndName(), - DB::ActionsDAG::MatchColumnsMode::Name); + DB::ActionsDAG::MatchColumnsMode::Name, + getContext()); LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Actions dag for replacing columns:\n{}", adjust_pos_actions_dag.dumpDAG()); auto adjust_pos_expression_step = std::make_unique(flatten_tuple_output_header, std::move(adjust_pos_actions_dag)); adjust_pos_expression_step->setStepDescription("Adjust position of the output columns"); @@ -455,8 +457,7 @@ void AggregateGroupLimitRelParser::addSortStep(DB::QueryPlan & plan) DB::SortingStep::Settings settings(getContext()->getSettingsRef()); auto config = MemoryConfig::loadFromContext(getContext()); - double spill_mem_ratio = config.spill_mem_ratio; - settings.worth_external_sort = [spill_mem_ratio]() -> bool { return currentThreadGroupMemoryUsageRatio() > spill_mem_ratio; }; + settings.worth_external_sort_mem_ratio = config.spill_mem_ratio; auto sorting_step = std::make_unique(plan.getCurrentHeader(), full_sort_descr, 0, settings); sorting_step->setStepDescription("Sorting step"); plan.addStep(std::move(sorting_step)); @@ -519,7 +520,7 @@ void AggregateGroupLimitRelParser::addWindowLimitStep(DB::QueryPlan & plan) win_descr.window_functions.push_back(win_func_description); auto win_step = std::make_unique(in_header, win_descr, win_descr.window_functions, false); - win_step->setStepDescription("Window (" + win_descr.window_name + ")"); + win_step->setStepDescription("Window (" + win_descr.window_name + ")", 1000); plan.addStep(std::move(win_step)); const auto & win_result_header = plan.getCurrentHeader(); diff --git a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp index db96600632c6..47615fe18dd9 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp @@ -172,7 +172,8 @@ void JoinRelParser::renamePlanColumns(DB::QueryPlan & left, DB::QueryPlan & righ ActionsDAG right_project = ActionsDAG::makeConvertingActions( right.getCurrentHeader()->getColumnsWithTypeAndName(), storage_join.getRightSampleBlock().getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Position); + ActionsDAG::MatchColumnsMode::Position, + getContext()); QueryPlanStepPtr right_project_step = std::make_unique(right.getCurrentHeader(), std::move(right_project)); right_project_step->setStepDescription("Rename Broadcast Table Name"); @@ -191,7 +192,7 @@ void JoinRelParser::renamePlanColumns(DB::QueryPlan & left, DB::QueryPlan & righ else new_left_cols.emplace_back(col.column, col.type, col.name); ActionsDAG left_project = ActionsDAG::makeConvertingActions( - left.getCurrentHeader()->getColumnsWithTypeAndName(), new_left_cols, ActionsDAG::MatchColumnsMode::Position); + left.getCurrentHeader()->getColumnsWithTypeAndName(), new_left_cols, ActionsDAG::MatchColumnsMode::Position, getContext()); QueryPlanStepPtr left_project_step = std::make_unique(left.getCurrentHeader(), std::move(left_project)); left_project_step->setStepDescription("Rename Left Table Name for broadcast join"); @@ -434,7 +435,7 @@ void JoinRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left, std::optional left_convert_actions; std::optional right_convert_actions; std::tie(left_convert_actions, right_convert_actions) = table_join.createConvertingActions( - left.getCurrentHeader()->getColumnsWithTypeAndName(), right.getCurrentHeader()->getColumnsWithTypeAndName()); + left.getCurrentHeader()->getColumnsWithTypeAndName(), right.getCurrentHeader()->getColumnsWithTypeAndName(), getContext()); if (right_convert_actions) { diff --git a/cpp-ch/local-engine/Parser/RelParsers/MergeTreeRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/MergeTreeRelParser.cpp index 8b5587201733..41208bdbaa30 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/MergeTreeRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/MergeTreeRelParser.cpp @@ -344,7 +344,8 @@ DB::QueryPlanPtr MergeTreeRelParser::parseReadRel( } auto read_step = storage->reader.readFromParts( - RangesInDataParts({selected_parts}), + std::make_shared(selected_parts), + // RangesInDataParts({selected_parts}), storage->getMutationsSnapshot({}), names_and_types_list.getNames(), storage_snapshot, @@ -596,7 +597,8 @@ String MergeTreeRelParser::filterRangesOnDriver(const substrait::ReadRel & read_ if (selected_parts.empty()) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "no data part found."); auto read_step = storage->reader.readFromParts( - RangesInDataParts({selected_parts}), + std::make_shared(selected_parts), + // RangesInDataParts({selected_parts}), /* alter_conversions = */ storage->getMutationsSnapshot({}), names_and_types_list.getNames(), diff --git a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp index 039084c6a5f5..ce964da04b9f 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp @@ -45,8 +45,7 @@ SortRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, st auto sort_descr = parseSortFields(*query_plan->getCurrentHeader(), sort_rel.sorts()); SortingStep::Settings settings(getContext()->getSettingsRef()); auto config = MemoryConfig::loadFromContext(getContext()); - double spill_mem_ratio = config.spill_mem_ratio; - settings.worth_external_sort = [spill_mem_ratio]() -> bool { return currentThreadGroupMemoryUsageRatio() > spill_mem_ratio; }; + settings.worth_external_sort_mem_ratio = config.spill_mem_ratio; auto sorting_step = std::make_unique(query_plan->getCurrentHeader(), sort_descr, limit, settings); sorting_step->setStepDescription("Sorting step"); steps.emplace_back(sorting_step.get()); diff --git a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp index 68791e80f859..4fb9feafff3e 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp @@ -85,7 +85,7 @@ WindowRelParser::parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & re auto & win = it.second; auto window_step = std::make_unique(current_plan->getCurrentHeader(), win, win.window_functions, false); - window_step->setStepDescription("Window step for window '" + win.window_name + "'"); + window_step->setStepDescription("Window step for window '" + win.window_name + "'", 1000); steps.emplace_back(window_step.get()); current_plan->addStep(std::move(window_step)); } @@ -332,7 +332,7 @@ void WindowRelParser::tryAddProjectionAfterWindow() if (!DB::blocksHaveEqualStructure(output_header, current_header)) { ActionsDAG convert_action = ActionsDAG::makeConvertingActions( - current_header.getColumnsWithTypeAndName(), output_header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Name); + current_header.getColumnsWithTypeAndName(), output_header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Name, getContext()); QueryPlanStepPtr convert_step = std::make_unique(current_plan->getCurrentHeader(), std::move(convert_action)); convert_step->setStepDescription("Convert window Output"); steps.emplace_back(convert_step.get()); diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp index ecf59f9fea96..aaa0779fe7bd 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp @@ -75,7 +75,7 @@ DB::ExpressionActionsPtr create_rename_action(const DB::Block & input, const DB: return std::make_shared(std::move(actions_dag)); } -DB::ExpressionActionsPtr create_project_action(const DB::Block & input, const DB::Block & output) +DB::ExpressionActionsPtr create_project_action(const DB::Block & input, const DB::Block & output, const DB::ContextPtr context) { DB::ColumnsWithTypeAndName final_cols; std::ranges::transform( @@ -89,11 +89,11 @@ DB::ExpressionActionsPtr create_project_action(const DB::Block & input, const DB assert(final_cols.size() == output.columns()); const auto & original_cols = input.getColumnsWithTypeAndName(); - ActionsDAG final_project = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); + ActionsDAG final_project = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position, context); return std::make_shared(std::move(final_project)); } -void adjust_output(const DB::QueryPipelineBuilderPtr & builder, const DB::Block & output) +void adjust_output(const DB::QueryPipelineBuilderPtr & builder, const DB::Block & output, const ContextPtr context) { const auto input = builder->getHeader(); if (input.columns() != output.columns()) @@ -111,7 +111,7 @@ void adjust_output(const DB::QueryPipelineBuilderPtr & builder, const DB::Block convert_action = create_rename_action(input, output); // name_is_different } else - convert_action = create_project_action(input, output); // type_is_different + convert_action = create_project_action(input, output, context); // type_is_different if (!convert_action) return; @@ -205,7 +205,7 @@ void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel { MergeTreeTable merge_tree_table(write, table_schema); auto output = toShared(TypeParser::buildBlockFromNamedStruct(table_schema, merge_tree_table.low_card_key)); - adjust_output(builder, *output); + adjust_output(builder, *output, context); builder->addSimpleTransform( [&](const SharedHeader & in_header) -> ProcessorPtr { return std::make_shared(in_header, false); }); @@ -224,7 +224,7 @@ void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel else { auto output = toShared(TypeParser::buildBlockFromNamedStruct(table_schema)); - adjust_output(builder, *output); + adjust_output(builder, *output, context); const auto partition_by = collect_partition_cols(*output, table_schema, partition_indexes); addNormalFileWriterSinkTransform(context, builder, write.common().format(), output, partition_by); } diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index c3cacdda8ab3..8c702e20922f 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -96,7 +96,7 @@ std::string join(const ActionsDAG::NodeRawConstPtrs & v, char c) return res; } -void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan) +void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan, const ContextPtr context) { const substrait::PlanRel & root_rel = plan.relations().at(0); if (root_rel.root().names_size()) @@ -180,7 +180,7 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con if (need_final_project) { ActionsDAG final_project - = ActionsDAG::makeConvertingActions(origin_columns, final_columns, ActionsDAG::MatchColumnsMode::Position, true); + = ActionsDAG::makeConvertingActions(origin_columns, final_columns, ActionsDAG::MatchColumnsMode::Position, context, true); QueryPlanStepPtr final_project_step = std::make_unique(query_plan->getCurrentHeader(), std::move(final_project)); final_project_step->setStepDescription("Project for output schema"); @@ -206,7 +206,7 @@ QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) std::list rel_stack; auto query_plan = parseOp(first_read_rel, rel_stack); if (!writePipeline) - adjustOutput(query_plan, plan); + adjustOutput(query_plan, plan, context); #ifndef NDEBUG PlanUtil::checkOuputType(*query_plan); @@ -313,7 +313,8 @@ DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPl CurrentThread::getGroup(), IAST::QueryKind::Select, settings, - 0); + 0, + false); QueryPlanOptimizationSettings optimization_settings{context}; // TODO: set optimize_plan to true when metrics could be collected while ch query plan optimization is enabled. diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index b237475da419..ded7d84360cb 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -117,7 +117,7 @@ class SerializedPlanParser private: DB::QueryPlanPtr parseOp(const substrait::Rel & rel, std::list & rel_stack); - static void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan); + static void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan, const DB::ContextPtr context); std::vector input_iters; std::vector split_infos; diff --git a/cpp-ch/local-engine/Parser/SparkRowToCHColumn.cpp b/cpp-ch/local-engine/Parser/SparkRowToCHColumn.cpp index 18417fb3b7e2..a6ff339ef9f0 100644 --- a/cpp-ch/local-engine/Parser/SparkRowToCHColumn.cpp +++ b/cpp-ch/local-engine/Parser/SparkRowToCHColumn.cpp @@ -55,11 +55,11 @@ ALWAYS_INLINE static void writeRowToColumns(const std::vector { if (spark_row_reader.supportRawData(i)) { - const StringRef str_ref{spark_row_reader.getStringRef(i)}; - if (str_ref.data == nullptr) + const std::string_view str_ref = spark_row_reader.getStringRef(i); + if (str_ref.data() == nullptr) columns[i]->insertDefault(); else if (!spark_row_reader.isBigEndianInSparkRow(i)) - columns[i]->insertData(str_ref.data, str_ref.size); + columns[i]->insertData(str_ref.data(), str_ref.size()); else columns[i]->insert(spark_row_reader.getField(i)); // read decimal128 } @@ -153,7 +153,7 @@ Field VariableLengthDataReader::read(const char * buffer, size_t length) const throw Exception(ErrorCodes::UNKNOWN_TYPE, "VariableLengthDataReader doesn't support type {}", type->getName()); } -StringRef VariableLengthDataReader::readUnalignedBytes(const char * buffer, size_t length) const +std::string_view VariableLengthDataReader::readUnalignedBytes(const char * buffer, size_t length) const { return {buffer, length}; } @@ -342,7 +342,7 @@ FixedLengthDataReader::FixedLengthDataReader(const DataTypePtr & type_) value_size = type_without_nullable->getSizeOfValueInMemory(); } -StringRef FixedLengthDataReader::unsafeRead(const char * buffer) const +std::string_view FixedLengthDataReader::unsafeRead(const char * buffer) const { return {buffer, value_size}; } diff --git a/cpp-ch/local-engine/Parser/SparkRowToCHColumn.h b/cpp-ch/local-engine/Parser/SparkRowToCHColumn.h index ee55a155691e..0e12a3876bf2 100644 --- a/cpp-ch/local-engine/Parser/SparkRowToCHColumn.h +++ b/cpp-ch/local-engine/Parser/SparkRowToCHColumn.h @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include @@ -139,7 +138,7 @@ class VariableLengthDataReader virtual ~VariableLengthDataReader() = default; virtual DB::Field read(const char * buffer, size_t length) const; - virtual StringRef readUnalignedBytes(const char * buffer, size_t length) const; + virtual std::string_view readUnalignedBytes(const char * buffer, size_t length) const; private: virtual DB::Field readDecimal(const char * buffer, size_t length) const; @@ -160,7 +159,7 @@ class FixedLengthDataReader virtual ~FixedLengthDataReader() = default; virtual DB::Field read(const char * buffer) const; - virtual StringRef unsafeRead(const char * buffer) const; + virtual std::string_view unsafeRead(const char * buffer) const; private: const DB::DataTypePtr type; @@ -294,13 +293,13 @@ class SparkRowReader return *reinterpret_cast(getFieldOffset(ordinal)); } - StringRef getString(size_t ordinal) const + std::string_view getString(size_t ordinal) const { assertIndexIsValid(ordinal); int64_t offset_and_size = getLong(ordinal); int32_t offset = static_cast(offset_and_size >> 32); int32_t size = static_cast(offset_and_size); - return StringRef(reinterpret_cast(this->buffer + offset), size); + return std::string_view(reinterpret_cast(this->buffer + offset), size); } int32_t getStringSize(size_t ordinal) const @@ -315,7 +314,7 @@ class SparkRowReader length = length_; } - StringRef getStringRef(size_t ordinal) const + std::string_view getStringRef(size_t ordinal) const { assertIndexIsValid(ordinal); if (!support_raw_datas[ordinal]) @@ -323,7 +322,7 @@ class SparkRowReader DB::ErrorCodes::UNKNOWN_TYPE, "SparkRowReader::getStringRef doesn't support type {}", field_types[ordinal]->getName()); if (isNullAt(ordinal)) - return StringRef(); + return std::string_view(); const auto & fixed_length_data_reader = fixed_length_data_readers[ordinal]; const auto & variable_length_data_reader = variable_length_data_readers[ordinal]; diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.cpp index 07b7541f1b1a..d7dd807edb88 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.cpp @@ -59,8 +59,7 @@ DB::ActionsDAG::NodeRawConstPtrs CountParser::parseFunctionArguments( auto uint8_type = std::make_shared(); DB::Field null_field; auto nullable_uint8_type = std::make_shared(uint8_type); - auto nullable_uint_col = nullable_uint8_type->createColumn(); - nullable_uint_col->insertDefault(); + auto nullable_uint_col = nullable_uint8_type->createColumnConst(1, nullable_uint8_type->getDefault()); const auto * const_1_node = &actions_dag.addColumn(DB::ColumnWithTypeAndName(uint8_type->createColumnConst(1, 1), uint8_type, getUniqueName("1"))); const auto * null_node diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.cpp index 5dc6a734b9a1..ee6092a4bae8 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.cpp @@ -43,6 +43,7 @@ LeadParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Act actions_dag, arg0_col, DB::makeNullable(arg0_col_type), + getContext(), arg0_col_name); actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); @@ -53,7 +54,7 @@ LeadParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Act } node = parseExpression(actions_dag, arg1); - node = ActionsDAGUtil::convertNodeType(actions_dag, node, BIGINT()); + node = ActionsDAGUtil::convertNodeType(actions_dag, node, BIGINT(), getContext()); actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); @@ -86,6 +87,7 @@ LagParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Acti actions_dag, arg0_col, makeNullable(arg0_col_type), + getContext(), arg0_col_name); actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); @@ -101,7 +103,7 @@ LagParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Acti auto real_field = 0 - literal_result.second.safeGet(); node = &actions_dag.addColumn(ColumnWithTypeAndName( literal_result.first->createColumnConst(1, real_field), literal_result.first, getUniqueName(toString(real_field)))); - node = ActionsDAGUtil::convertNodeType(actions_dag, node, BIGINT()); + node = ActionsDAGUtil::convertNodeType(actions_dag, node, BIGINT(), getContext()); actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/PercentileParserBase.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/PercentileParserBase.cpp index ceffc18afecc..9a56222ac4b9 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/PercentileParserBase.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/PercentileParserBase.cpp @@ -129,7 +129,7 @@ DB::Array PercentileParserBase::parseFunctionParameters( if (arg_nodes[i]->result_type->isNullable()) dst_type = std::make_shared(dst_type); - arg_nodes[i] = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, arg_nodes[i], dst_type); + arg_nodes[i] = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, arg_nodes[i], dst_type, getContext()); } new_arg_nodes.emplace_back(arg_nodes[i]); diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/SimpleStatisticsFunctions.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/SimpleStatisticsFunctions.cpp index a7acd18d26ce..bda181e0fad8 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/SimpleStatisticsFunctions.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/SimpleStatisticsFunctions.cpp @@ -53,8 +53,7 @@ class AggregateFunctionParserStddev final : public AggregateFunctionParser /// if result is NaN, convert it to NULL. auto is_nan_func_node = toFunctionNode(actions_dag, "isNaN", getUniqueName("isNaN"), {func_node}); auto null_type = DB::makeNullable(func_node->result_type); - auto nullable_col = null_type->createColumn(); - nullable_col->insertDefault(); + auto nullable_col = null_type->createColumnConst(1, null_type->getDefault()); const auto * null_node = &actions_dag.addColumn(DB::ColumnWithTypeAndName(std::move(nullable_col), null_type, getUniqueName("null"))); DB::ActionsDAG::NodeRawConstPtrs convert_nan_func_args = {is_nan_func_node, null_node, func_node}; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp index 5248a811da3d..22fefa5e6a59 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp @@ -63,7 +63,7 @@ class FunctionParserArrayFilter : public FunctionParser /// filter with index argument. const auto * range_end_node = toFunctionNode(actions_dag, "length", {toFunctionNode(actions_dag, "assumeNotNull", {parsed_args[0]})}); range_end_node = ActionsDAGUtil::convertNodeType( - actions_dag, range_end_node, makeNullable(INT()), range_end_node->result_name); + actions_dag, range_end_node, makeNullable(INT()), getContext(), range_end_node->result_name); const auto * index_array_node = toFunctionNode( actions_dag, "range", @@ -102,14 +102,14 @@ class FunctionParserArrayTransform : public FunctionParser DataTypePtr dst_array_type = std::make_shared(lambda_args.front().type); if (src_array_type->isNullable()) dst_array_type = std::make_shared(dst_array_type); - const auto * dst_array_arg = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, parsed_args[0], dst_array_type); + const auto * dst_array_arg = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, parsed_args[0], dst_array_type, getContext()); return toFunctionNode(actions_dag, ch_func_name, {parsed_args[1], dst_array_arg}); } /// transform with index argument. const auto * range_end_node = toFunctionNode(actions_dag, "length", {toFunctionNode(actions_dag, "assumeNotNull", {parsed_args[0]})}); range_end_node = ActionsDAGUtil::convertNodeType( - actions_dag, range_end_node, makeNullable(INT()), range_end_node->result_name); + actions_dag, range_end_node, makeNullable(INT()), getContext(), range_end_node->result_name); const auto * index_array_node = toFunctionNode( actions_dag, "range", @@ -128,35 +128,57 @@ class FunctionParserArrayAggregate : public FunctionParser String getName() const override { return name; } String getCHFunctionName(const substrait::Expression_ScalarFunction & scalar_function) const override { - return "arrayFold"; + return "sparkArrayFold"; } const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { auto ch_func_name = getCHFunctionName(substrait_func); auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); - assert(parsed_args.size() == 3); + if (parsed_args.size() != 3 && parsed_args.size() != 4) + throw DB::Exception(DB::ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "aggregate function must have three or four arguments"); - const auto * function_type = typeid_cast(parsed_args[2]->result_type.get()); - if (!function_type) + const auto * merge_function_type = typeid_cast(parsed_args[2]->result_type.get()); + if (!merge_function_type) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "The third argument of aggregate function must be a lambda function"); - if (!parsed_args[1]->result_type->equals(*(function_type->getReturnType()))) + const auto & merge_arg_types = merge_function_type->getArgumentTypes(); + if (merge_arg_types.size() != 2) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "The third argument of aggregate function must be a lambda function with 2 arguments"); + + if (!parsed_args[1]->result_type->equals(*(merge_arg_types.front()))) { parsed_args[1] = ActionsDAGUtil::convertNodeType( actions_dag, parsed_args[1], - function_type->getReturnType(), + merge_arg_types.front(), + getContext(), parsed_args[1]->result_name); } + if (parsed_args.size() == 4) + { + const auto * finish_function_type = typeid_cast(parsed_args[3]->result_type.get()); + if (!finish_function_type || finish_function_type->getArgumentTypes().size() != 1) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "The fourth argument of aggregate function must be a lambda function"); + } + + /// Align array element type with merge lambda argument type. + const auto & merge_element_type = merge_arg_types.back(); + const auto & src_array_type = parsed_args[0]->result_type; + DataTypePtr dst_array_type = std::make_shared(merge_element_type); + if (src_array_type->isNullable()) + dst_array_type = std::make_shared(dst_array_type); + const auto * array_col_node = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, parsed_args[0], dst_array_type, getContext()); + /// arrayFold cannot accept nullable(array) - const auto * array_col_node = parsed_args[0]; if (parsed_args[0]->result_type->isNullable()) { array_col_node = toFunctionNode(actions_dag, "assumeNotNull", {parsed_args[0]}); } - const auto * func_node = toFunctionNode(actions_dag, ch_func_name, {parsed_args[2], array_col_node, parsed_args[1]}); + const auto * func_node = parsed_args.size() == 4 + ? toFunctionNode(actions_dag, ch_func_name, {parsed_args[2], array_col_node, parsed_args[1], parsed_args[3]}) + : toFunctionNode(actions_dag, ch_func_name, {parsed_args[2], array_col_node, parsed_args[1]}); /// For null array, result is null. /// TODO: make a new version of arrayFold that can handle nullable array. const auto * is_null_node = toFunctionNode(actions_dag, "isNull", {parsed_args[0]}); @@ -205,7 +227,7 @@ class FunctionParserArraySort : public FunctionParser DataTypePtr dst_array_type = std::make_shared(lambda_args.front().type); if (src_array_type->isNullable()) dst_array_type = std::make_shared(dst_array_type); - parsed_args[0] = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, parsed_args[0], dst_array_type); + parsed_args[0] = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, parsed_args[0], dst_array_type, getContext()); return toFunctionNode(actions_dag, ch_func_name, {parsed_args[1], parsed_args[0]}); } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp index a901b4ca4ada..6d094d39af09 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp @@ -86,7 +86,7 @@ class FunctionParserArrayPosition : public FunctionParser DataTypePtr wrap_arr_nullable_type = wrapNullableType(true, ch_function_node->result_type); const auto * wrap_index_of_node = ActionsDAGUtil::convertNodeType( - actions_dag, ch_function_node, wrap_arr_nullable_type, ch_function_node->result_name); + actions_dag, ch_function_node, wrap_arr_nullable_type, getContext(), ch_function_node->result_name); const auto * null_const_node = addColumnToActionsDAG(actions_dag, wrap_arr_nullable_type, Field{}); const auto * or_condition_node = toFunctionNode(actions_dag, "or", {arr_is_null_node, val_is_null_node}); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/elt.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/elt.cpp index 0657ea9b69ec..a8012bf43754 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/elt.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/elt.cpp @@ -75,7 +75,7 @@ class FunctionParserElt : public FunctionParser auto nullable_result_type = makeNullable(result_type); const auto * nullable_array_element_node = ActionsDAGUtil::convertNodeType( - actions_dag, array_element_node, nullable_result_type, array_element_node->result_name); + actions_dag, array_element_node, nullable_result_type, getContext(), array_element_node->result_name); const auto * null_const_node = addColumnToActionsDAG(actions_dag, nullable_result_type, DB::Field()); const auto * is_null_node = toFunctionNode(actions_dag, "isNull", {index_arg}); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/findInset.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/findInset.cpp index fd43356674b6..5b7d58ed420f 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/findInset.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/findInset.cpp @@ -75,7 +75,7 @@ class FunctionParserFindInSet : public FunctionParser auto nullable_result_type = makeNullable(INT()); const auto * nullable_index_of_node = ActionsDAGUtil::convertNodeType( - actions_dag, index_of_node, nullable_result_type, index_of_node->result_name); + actions_dag, index_of_node, nullable_result_type, getContext(), index_of_node->result_name); const auto * null_const_node = addColumnToActionsDAG(actions_dag, nullable_result_type, DB::Field()); const auto * str_is_null_node = toFunctionNode(actions_dag, "isNull", {str_arg}); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h index f5c3ac0d713f..ed97f644d7e6 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h @@ -77,7 +77,7 @@ class FunctionParserGetTimestamp : public FunctionParser throw DB::Exception(DB::ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The second argument of function {} must be const String.", name); UInt32 s_count = std::count(fmt.begin(), fmt.end(), 'S'); - String time_parser_policy = getContext()->getSettingsRef().has(TIMER_PARSER_POLICY) ? toString(getContext()->getSettingsRef().get(TIMER_PARSER_POLICY)) : ""; + String time_parser_policy = getContext()->getSettingsRef().has(TIMER_PARSER_POLICY) ? DB::fieldToString(getContext()->getSettingsRef().get(TIMER_PARSER_POLICY)) : ""; boost::to_lower(time_parser_policy); if (time_parser_policy == "legacy" && checkFormat(fmt)) { diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp index 1ed40643f654..0126af14a03b 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp @@ -52,7 +52,7 @@ class FunctionParserLocate : public FunctionParser const auto * substr_arg = parsed_args[0]; const auto * str_arg = parsed_args[1]; - const auto * start_pos_arg = ActionsDAGUtil::convertNodeType(actions_dag, parsed_args[2], makeNullable(UINT())); + const auto * start_pos_arg = ActionsDAGUtil::convertNodeType(actions_dag, parsed_args[2], makeNullable(UINT()), getContext()); const auto * is_start_pos_null_node = toFunctionNode(actions_dag, "isNull", {start_pos_arg}); const auto * const_1_node = addColumnToActionsDAG(actions_dag, std::make_shared(), 0); const auto * position_node = toFunctionNode(actions_dag, "positionUTF8Spark", {str_arg, substr_arg, start_pos_arg}); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp index a77ca7699cd0..a2ba088415ff 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp @@ -68,7 +68,7 @@ class FunctionParserMapTransformImpl : public FunctionParser auto array_type = checkAndGetDataType(removeNullable(map_type).get())->getNestedType(); if (map_type->isNullable()) array_type = std::make_shared(array_type); - const auto * array_node = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, map_node, array_type); + const auto * array_node = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, map_node, array_type, getContext()); const auto * transformed_node = toFunctionNode(actions_dag, "arrayMap", {func_node, array_node}); const ActionsDAG::Node * result_node = nullptr; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp index c6a20cd43a32..432e676679b0 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp @@ -90,7 +90,7 @@ class FunctionParserArraySlice : public FunctionParser DB::DataTypePtr wrap_arr_nullable_type = wrapNullableType(true, slice_node->result_type); const auto * wrap_slice_node = ActionsDAGUtil::convertNodeType( - actions_dag, slice_node, wrap_arr_nullable_type, slice_node->result_name); + actions_dag, slice_node, wrap_arr_nullable_type, getContext(), slice_node->result_name); const auto * null_const_node = addColumnToActionsDAG(actions_dag, wrap_arr_nullable_type, DB::Field{}); const auto * arr_is_null_node = toFunctionNode(actions_dag, "isNull", {arr_arg}); diff --git a/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp index b1b9c596839c..7d6a599569de 100644 --- a/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp +++ b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include @@ -122,7 +122,7 @@ Task CacheManager::cachePart( } auto query_info = buildQueryInfo(names_and_types_list); auto read_step = storage->reader.readFromParts( - RangesInDataParts({selected_parts}), + std::make_shared(selected_parts), storage->getMutationsSnapshot({}), names_and_types_list.getNames(), storage_snapshot, diff --git a/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.cpp b/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.cpp index a7501d6a2820..89693eacc867 100644 --- a/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.cpp +++ b/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.cpp @@ -114,7 +114,7 @@ DB::ColumnWithTypeAndName convertFixedStringToAggregateState(const DB::ColumnWit agg_function->create(place); auto value = col.column->getDataAt(i); - memcpy(place, value.data, value.size); + memcpy(place, value.data(), value.size()); vec.push_back(place); } diff --git a/cpp-ch/local-engine/Storages/IO/NativeReader.cpp b/cpp-ch/local-engine/Storages/IO/NativeReader.cpp index 524cb3af3835..5b758b887641 100644 --- a/cpp-ch/local-engine/Storages/IO/NativeReader.cpp +++ b/cpp-ch/local-engine/Storages/IO/NativeReader.cpp @@ -125,7 +125,15 @@ readNormalSimpleData(DB::ReadBuffer & in, DB::ColumnPtr & column, size_t rows, N { ISerialization::DeserializeBinaryBulkSettings settings; settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return ∈ }; - settings.avg_value_size_hint = column_parse_util.avg_value_size_hint; + // settings.avg_value_size_hint = column_parse_util.avg_value_size_hint; + settings.get_avg_value_size_hint_callback = [&column_parse_util](const ISerialization::SubstreamPath & /* substream_path */) -> double + { + return column_parse_util.avg_value_size_hint; + }; + settings.update_avg_value_size_hint_callback = [&column_parse_util](const ISerialization::SubstreamPath &, const IColumn &) -> void + { + // TODO: rebase-25.12, how to update + }; settings.position_independent_encoding = false; settings.native_format = true; @@ -141,7 +149,15 @@ readNormalComplexData(DB::ReadBuffer & in, DB::ColumnPtr & column, size_t rows, { ISerialization::DeserializeBinaryBulkSettings settings; settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return ∈ }; - settings.avg_value_size_hint = column_parse_util.avg_value_size_hint; + // settings.avg_value_size_hint = column_parse_util.avg_value_size_hint; + settings.get_avg_value_size_hint_callback = [&column_parse_util](const ISerialization::SubstreamPath & /* substream_path */) -> double + { + return column_parse_util.avg_value_size_hint; + }; + settings.update_avg_value_size_hint_callback = [&column_parse_util](const ISerialization::SubstreamPath &, const IColumn &) -> void + { + // TODO: rebase-25.12, how to update + }; settings.position_independent_encoding = false; settings.native_format = true; diff --git a/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp b/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp index c19005cd13dd..c9d95e9f7f00 100644 --- a/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp +++ b/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp @@ -43,7 +43,7 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) */ - ColumnPtr full_column = column->convertToFullColumnIfConst(); + ColumnPtr full_column = column->convertToFullColumnIfReplicated()->convertToFullColumnIfConst(); ISerialization::SerializeBinaryBulkSettings settings; settings.getter = [&ostr](ISerialization::SubstreamPath) -> WriteBuffer * { return &ostr; }; diff --git a/cpp-ch/local-engine/Storages/Kafka/GlutenKafkaSource.cpp b/cpp-ch/local-engine/Storages/Kafka/GlutenKafkaSource.cpp index 0abebbd970f1..01ff8c0f6840 100644 --- a/cpp-ch/local-engine/Storages/Kafka/GlutenKafkaSource.cpp +++ b/cpp-ch/local-engine/Storages/Kafka/GlutenKafkaSource.cpp @@ -310,7 +310,8 @@ Chunk GlutenKafkaSource::generateImpl() auto converting_dag = ActionsDAG::makeConvertingActions( result_block.cloneEmpty().getColumnsWithTypeAndName(), getPort().getHeader().getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); + ActionsDAG::MatchColumnsMode::Name, + context); auto converting_actions = std::make_shared(std::move(converting_dag)); converting_actions->execute(result_block); diff --git a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp index b10a382b89d1..4fb733f2a733 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp @@ -25,6 +25,7 @@ #include #include #include +#include namespace DB { @@ -58,7 +59,7 @@ bool MergeSparkMergeTreeTask::executeStep() std::optional switcher; if (merge_list_entry) { - switcher.emplace((*merge_list_entry)->thread_group, "", /*allow_existing_group*/ true); + switcher.emplace((*merge_list_entry)->thread_group, ThreadName::GLUTEN_MERGE_MERGETREE, /*allow_existing_group*/ true); } switch (state) diff --git a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp index d55c57d1956e..74d928cd812d 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp @@ -17,9 +17,10 @@ #include "MetaDataHelper.h" #include #include -#include +#include #include #include +#include #include #include @@ -119,7 +120,7 @@ void restoreMetaData( return; else transaction->createDirectoryRecursive(part_path); - auto key = s3->generateObjectKeyForPath(metadata_file_path.generic_string(), std::nullopt); + auto key = transaction->generateObjectKeyForPath(metadata_file_path.generic_string()); StoredObject metadata_object(key.serialize()); auto read_settings = ReadSettings{}; read_settings.enable_filesystem_cache = false; @@ -172,6 +173,8 @@ void restoreMetaData( if (!metadata_disk->existsDirectory(table_path)) metadata_disk->createDirectories(table_path.generic_string()); + auto transaction = data_disk->getMetadataStorage()->createTransaction(); + for (const auto & part : not_exists_part) { auto job = [&]() @@ -183,7 +186,8 @@ void restoreMetaData( return; else metadata_disk->createDirectories(part_path); - auto key = s3->generateObjectKeyForPath(metadata_file_path.generic_string(), std::nullopt); + // TODO: rebase-25.12, fix later, how to call 'generateObjectKeyForPath' + auto key = transaction->generateObjectKeyForPath(metadata_file_path.generic_string()); StoredObject metadata_object(key.serialize()); auto read_settings = ReadSettings{}; read_settings.enable_filesystem_cache = false; @@ -256,15 +260,19 @@ MergeTreeDataPartPtr mergeParts( auto future_part = std::make_shared(); future_part->uuid = UUIDHelpers::generateV4(); - future_part->assign(std::move(selected_parts), /*patch_parts_=*/ {}); + // TODO: rebase-25.12, how to assign the projection parameter + future_part->assign(std::move(selected_parts), /*patch_parts_=*/ {}, nullptr); future_part->part_info = MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION; - //TODO: name + // TODO: name future_part->name = partition_dir.empty() ? "" : partition_dir + "/"; if (!bucket_dir.empty()) future_part->name = future_part->name + bucket_dir + "/"; future_part->name = future_part->name + new_part_uuid + "-merged"; + // TODO: rebase-25.12, how to generate CurrentlyMergingPartsTagger + // uint64_t needed_disk_space = CompactionStatistics::estimateNeededDiskSpace(future_part->parts, true); + // auto tagger = std::make_unique(future_part, needed_disk_space, NULL, nullptr, false); auto entry = std::make_shared( future_part, DB::CurrentlyMergingPartsTaggerPtr{}, std::make_shared()); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp index 8e24c271d1e7..80e361874a36 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp @@ -22,6 +22,7 @@ #include #include #include +#include namespace CurrentMetrics { @@ -177,7 +178,7 @@ void SinkHelper::doMergePartsAsync(const std::vector & merge_part thread_pool.scheduleOrThrow( [this, merge_parts_with_stats, thread_group = CurrentThread::getGroup()]() -> void { - ThreadGroupSwitcher switcher(thread_group, "AsyncMerge"); + ThreadGroupSwitcher switcher(thread_group, ThreadName::ASYNC_MERGE); Stopwatch watch; size_t before_size = 0; diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp index 2ec73a5611ac..eea676a96040 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp @@ -119,12 +119,12 @@ std::unique_ptr SparkMergeTreeWriter::create( // // auto stats = std::make_shared(header, sink_helper); // chain.addSink(stats); - return std::make_unique(header, sink_helper, QueryPipeline{std::move(chain)}, spark_job_id); + return std::make_unique(header, sink_helper, QueryPipeline{std::move(chain)}, spark_job_id, context); } SparkMergeTreeWriter::SparkMergeTreeWriter( - const DB::Block & header_, const SinkHelper & sink_helper_, DB::QueryPipeline && pipeline_, const std::string & spark_job_id_) - : header{header_}, sink_helper{sink_helper_}, pipeline{std::move(pipeline_)}, executor{pipeline}, spark_job_id(spark_job_id_) + const DB::Block & header_, const SinkHelper & sink_helper_, DB::QueryPipeline && pipeline_, const std::string & spark_job_id_, const DB::ContextPtr & context_) + : header{header_}, sink_helper{sink_helper_}, pipeline{std::move(pipeline_)}, executor{pipeline}, spark_job_id(spark_job_id_), context(context_) { } @@ -132,7 +132,7 @@ void SparkMergeTreeWriter::write(const DB::Block & block) { auto new_block = removeColumnSuffix(block); auto converter = ActionsDAG::makeConvertingActions( - new_block.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Position); + new_block.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Position, context); const ExpressionActions expression_actions{std::move(converter)}; expression_actions.execute(new_block); executor.push(new_block); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h index 93ea48f5a140..93093bab4387 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h @@ -56,7 +56,7 @@ class SparkMergeTreeWriter : public NativeOutputWriter create(const MergeTreeTable & merge_tree_table, const DB::ContextMutablePtr & context, const std::string & spark_job_id); SparkMergeTreeWriter( - const DB::Block & header_, const SinkHelper & sink_helper_, DB::QueryPipeline && pipeline_, const std::string & spark_job_id_); + const DB::Block & header_, const SinkHelper & sink_helper_, DB::QueryPipeline && pipeline_, const std::string & spark_job_id_, const DB::ContextPtr & context_); void write(const DB::Block & block) override; void close() override; @@ -70,6 +70,7 @@ class SparkMergeTreeWriter : public NativeOutputWriter DB::QueryPipeline pipeline; DB::PushingPipelineExecutor executor; const std::string spark_job_id; + const DB::ContextPtr context; std::vector getAllPartInfo() const; }; diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index 8ab29e7e8150..bee333b73bd2 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -16,6 +16,7 @@ */ #include "SparkStorageMergeTree.h" +#include #include #include #include @@ -33,6 +34,10 @@ extern const Event LoadedDataPartsMicroseconds; namespace DB { +namespace Setting +{ +extern const SettingsBool optimize_on_insert; +} namespace MergeTreeSetting { extern const MergeTreeSettingsBool assign_part_uuids; @@ -290,7 +295,7 @@ MergeTreeData::LoadPartResult SparkStorageMergeTree::loadDataPart( // without it "test mergetree optimize partitioned by one low card column" will log ERROR resetColumnSizes(); - calculateColumnAndSecondaryIndexSizesImpl(); + calculateColumnAndSecondaryIndexSizesImpl(parts_lock); LOG_TRACE(log, "Finished loading {} part {} on disk {}", magic_enum::enum_name(to_state), part_name, part_disk_ptr->getName()); return res; @@ -299,7 +304,7 @@ MergeTreeData::LoadPartResult SparkStorageMergeTree::loadDataPart( void SparkStorageMergeTree::removePartFromMemory(const MergeTreeData::DataPart & part_to_detach) { auto lock = lockParts(); - bool removed_active_part = false; + // bool removed_active_part = false; bool restored_active_part = false; auto it_part = data_parts_by_info.find(part_to_detach.info); @@ -317,15 +322,16 @@ void SparkStorageMergeTree::removePartFromMemory(const MergeTreeData::DataPart & if (part->getState() == DataPartState::Active) { removePartContributionToColumnAndSecondaryIndexSizes(part); - removed_active_part = true; + // removed_active_part = true; } - modifyPartState(it_part, DataPartState::Deleting); + modifyPartState(it_part, DataPartState::Deleting, lock); LOG_TEST(log, "removePartFromMemory: removing {} from data_parts_indexes", part->getNameWithState()); data_parts_indexes.erase(it_part); - if (removed_active_part || restored_active_part) - resetObjectColumnsFromActiveParts(lock); + // TODO: rebase-25.12, remove first, the function 'resetObjectColumnsFromActiveParts' has been removed. + // if (removed_active_part || restored_active_part) + // resetObjectColumnsFromActiveParts(lock); } void SparkStorageMergeTree::dropPartNoWaitNoThrow(const String & /*part_name*/) @@ -402,9 +408,15 @@ MergeTreeTemporaryPartPtr SparkMergeTreeDataWriter::writeTempPart( auto minmax_idx = std::make_shared(); minmax_idx->update(block, MergeTreeData::getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); + const auto & global_settings = context->getSettingsRef(); + const MergeTreeSettingsPtr & data_settings = data.getSettings(); + bool optimize_on_insert = global_settings[Setting::optimize_on_insert] + && data.merging_params.mode != MergeTreeData::MergingParams::Ordinary; + UInt32 new_part_level = optimize_on_insert ? 1 : 0; + MergeTreePartition partition(block_with_partition.partition); - MergeTreePartInfo new_part_info(partition.getID(metadata_snapshot->getPartitionKey().sample_block), 1, 1, 0); + MergeTreePartInfo new_part_info(partition.getID(metadata_snapshot->getPartitionKey().sample_block), 1, 1, new_part_level); temp_part->temporary_directory_lock = data.getTemporaryPartDirectoryHolder(part_dir); @@ -447,16 +459,17 @@ MergeTreeTemporaryPartPtr SparkMergeTreeDataWriter::writeTempPart( VolumePtr volume = data.getStoragePolicy()->getVolume(0); VolumePtr data_part_volume = std::make_shared(volume->getName(), volume->getDisk(), volume->max_data_part_size); auto new_data_part = data.getDataPartBuilder(part_dir, data_part_volume, part_dir, context->getReadSettings()) - .withPartFormat(data.choosePartFormat(expected_size, block.rows())) + .withPartFormat(data.choosePartFormat(expected_size, block.rows(), new_part_level, nullptr)) .withPartInfo(new_part_info) .build(); auto data_part_storage = new_data_part->getDataPartStoragePtr(); - - const MergeTreeSettings & data_settings = *data.getSettings(); - - SerializationInfo::Settings settings{data_settings[MergeTreeSetting::ratio_of_defaults_for_sparse_serialization], true}; + // TODO: rebase-25.12, the versions are right? + MergeTreeSerializationInfoVersion version = MergeTreeSerializationInfoVersion::BASIC; + MergeTreeStringSerializationVersion string_serialization_version = MergeTreeStringSerializationVersion::SINGLE_STREAM; + MergeTreeNullableSerializationVersion nullable_serialization_version = MergeTreeNullableSerializationVersion::BASIC; + SerializationInfo::Settings settings{(*data_settings)[MergeTreeSetting::ratio_of_defaults_for_sparse_serialization], true, version, string_serialization_version, nullable_serialization_version}; SerializationInfoByName infos(columns, settings); infos.add(block); @@ -467,7 +480,7 @@ MergeTreeTemporaryPartPtr SparkMergeTreeDataWriter::writeTempPart( data_part_storage->beginTransaction(); - if (data_settings[MergeTreeSetting::assign_part_uuids]) + if ((*data_settings)[MergeTreeSetting::assign_part_uuids]) new_data_part->uuid = UUIDHelpers::generateV4(); SyncGuardPtr sync_guard; @@ -483,7 +496,7 @@ MergeTreeTemporaryPartPtr SparkMergeTreeDataWriter::writeTempPart( data_part_storage->createDirectories(); - if ((*data.getSettings())[MergeTreeSetting::fsync_part_directory]) + if ((*data_settings)[MergeTreeSetting::fsync_part_directory]) { const auto disk = data_part_volume->getDisk(); sync_guard = disk->getDirectorySyncGuard(full_path); @@ -501,6 +514,7 @@ MergeTreeTemporaryPartPtr SparkMergeTreeDataWriter::writeTempPart( /*blocks_are_granules=*/false); auto out = std::make_unique( new_data_part, + data_settings, metadata_snapshot, columns, indices, @@ -514,7 +528,7 @@ MergeTreeTemporaryPartPtr SparkMergeTreeDataWriter::writeTempPart( context->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); - auto finalizer = out->finalizePartAsync(new_data_part, data_settings[MergeTreeSetting::fsync_after_insert], nullptr, nullptr); + auto finalizer = out->finalizePartAsync(new_data_part, (*data_settings)[MergeTreeSetting::fsync_after_insert], nullptr, nullptr); temp_part->part = new_data_part; temp_part->streams.emplace_back(MergeTreeTemporaryPart::Stream{.stream = std::move(out), .finalizer = std::move(finalizer)}); diff --git a/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.cpp b/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.cpp index f20a9b88ad81..e994af53d768 100644 --- a/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.cpp +++ b/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.cpp @@ -724,71 +724,71 @@ ColumnIndexPtr ColumnIndex::create( /// const ColumnIndexFilter::AtomMap ColumnIndexFilter::atom_map{ {"notEquals", - [](RPNElement & out, const DB::Field & value) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field & value) { - out.function = RPNElement::FUNCTION_NOT_EQUALS; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_NOT_EQUALS; out.value = value; return true; }}, {"equals", - [](RPNElement & out, const DB::Field & value) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field & value) { - out.function = RPNElement::FUNCTION_EQUALS; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_EQUALS; out.value = value; return true; }}, {"less", - [](RPNElement & out, const DB::Field & value) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field & value) { - out.function = RPNElement::FUNCTION_LESS; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_LESS; out.value = value; return true; }}, {"greater", - [](RPNElement & out, const DB::Field & value) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field & value) { - out.function = RPNElement::FUNCTION_GREATER; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_GREATER; out.value = value; return true; }}, {"lessOrEquals", - [](RPNElement & out, const DB::Field & value) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field & value) { - out.function = RPNElement::FUNCTION_LESS_OR_EQUALS; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_LESS_OR_EQUALS; out.value = value; return true; }}, {"greaterOrEquals", - [](RPNElement & out, const DB::Field & value) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field & value) { - out.function = RPNElement::FUNCTION_GREATER_OR_EQUALS; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_GREATER_OR_EQUALS; out.value = value; return true; }}, {"in", - [](RPNElement & out, const DB::Field &) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field &) { - out.function = RPNElement::FUNCTION_IN; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_IN; return true; }}, {"notIn", - [](RPNElement & out, const DB::Field &) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field &) { - out.function = RPNElement::FUNCTION_NOT_IN; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_NOT_IN; return true; }}, {"isNotNull", - [](RPNElement & out, const DB::Field &) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field &) { /// Field's default constructor constructs a null value - out.function = RPNElement::FUNCTION_NOT_EQUALS; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_NOT_EQUALS; return true; }}, {"isNull", - [](RPNElement & out, const DB::Field &) + [](DB::GlutenParquetColumnIndexFilter::RPNElement & out, const DB::Field &) { /// Field's default constructor constructs a null value - out.function = RPNElement::FUNCTION_EQUALS; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_EQUALS; return true; }}}; @@ -797,14 +797,14 @@ ColumnIndexFilter::ColumnIndexFilter(const DB::ActionsDAG & filter_dag, DB::Cont DB::ActionsDAGWithInversionPushDown inverted_dag(filter_dag.getOutputs().front(), context); assert(inverted_dag.predicate != nullptr); - DB::RPNBuilder builder( + DB::RPNBuilder builder( inverted_dag.predicate, std::move(context), - [&](const DB::RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); + [&](const DB::RPNBuilderTreeNode & node, DB::GlutenParquetColumnIndexFilter::RPNElement & out) { return extractAtomFromTree(node, out); }); rpn_ = std::move(builder).extractRPN(); } -bool tryPrepareSetIndex(const DB::RPNBuilderFunctionTreeNode & func, ColumnIndexFilter::RPNElement & out) +bool tryPrepareSetIndex(const DB::RPNBuilderFunctionTreeNode & func, DB::GlutenParquetColumnIndexFilter::RPNElement & out) { const auto right_arg = func.getArgumentAt(1); const auto future_set = right_arg.tryGetPreparedSet(); @@ -829,7 +829,7 @@ bool tryPrepareSetIndex(const DB::RPNBuilderFunctionTreeNode & func, ColumnIndex return true; } -bool ColumnIndexFilter::extractAtomFromTree(const DB::RPNBuilderTreeNode & node, RPNElement & out) +bool ColumnIndexFilter::extractAtomFromTree(const DB::RPNBuilderTreeNode & node, DB::GlutenParquetColumnIndexFilter::RPNElement & out) { DB::Field const_value; DB::DataTypePtr const_type; @@ -857,7 +857,7 @@ bool ColumnIndexFilter::extractAtomFromTree(const DB::RPNBuilderTreeNode & node, /// If the const operand is null, the atom will be always false if (const_value.isNull()) { - out.function = RPNElement::ALWAYS_FALSE; + out.function = DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_FALSE; return true; } out.columnName = func.getArgumentAt(0).getColumnName(); @@ -879,17 +879,17 @@ bool ColumnIndexFilter::extractAtomFromTree(const DB::RPNBuilderTreeNode & node, /// For cases where it says, for example, `WHERE 0 AND something` if (const_value.getType() == DB::Field::Types::UInt64) { - out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_TRUE : DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_FALSE; return true; } else if (const_value.getType() == DB::Field::Types::Int64) { - out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_TRUE : DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_FALSE; return true; } else if (const_value.getType() == DB::Field::Types::Float64) { - out.function = const_value.safeGet() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() != 0.0 ? DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_TRUE : DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_FALSE; return true; } } @@ -899,7 +899,7 @@ bool ColumnIndexFilter::extractAtomFromTree(const DB::RPNBuilderTreeNode & node, RowRanges ColumnIndexFilter::calculateRowRanges(const ColumnIndexStore & index_store, size_t rowgroup_count) const { using LOGICAL_OP = RowRanges (*)(const RowRanges &, const RowRanges &); - using OPERATOR = std::function; + using OPERATOR = std::function; std::vector rpn_stack; auto CALL_LOGICAL_OP = [&rpn_stack](const LOGICAL_OP & op) @@ -911,7 +911,7 @@ RowRanges ColumnIndexFilter::calculateRowRanges(const ColumnIndexStore & index_s rpn_stack.back() = op(arg1, arg2); }; - auto CALL_OPERATOR = [&rpn_stack, &index_store, rowgroup_count](const RPNElement & element, const OPERATOR & callback) + auto CALL_OPERATOR = [&rpn_stack, &index_store, rowgroup_count](const DB::GlutenParquetColumnIndexFilter::RPNElement & element, const OPERATOR & callback) { const auto it = index_store.find(element.columnName); if (it != index_store.end() && it->second->hasParquetColumnIndex()) @@ -938,48 +938,48 @@ RowRanges ColumnIndexFilter::calculateRowRanges(const ColumnIndexStore & index_s { switch (element.function) { - case RPNElement::FUNCTION_EQUALS: - CALL_OPERATOR(element, [](const ColumnIndex & index, const RPNElement & e) { return index.eq(e.value); }); + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_EQUALS: + CALL_OPERATOR(element, [](const ColumnIndex & index, const DB::GlutenParquetColumnIndexFilter::RPNElement & e) { return index.eq(e.value); }); break; - case RPNElement::FUNCTION_NOT_EQUALS: - CALL_OPERATOR(element, [](const ColumnIndex & index, const RPNElement & e) { return index.notEq(e.value); }); + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_NOT_EQUALS: + CALL_OPERATOR(element, [](const ColumnIndex & index, const DB::GlutenParquetColumnIndexFilter::RPNElement & e) { return index.notEq(e.value); }); break; - case RPNElement::FUNCTION_LESS: - CALL_OPERATOR(element, [](const ColumnIndex & index, const RPNElement & e) { return index.lt(e.value); }); + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_LESS: + CALL_OPERATOR(element, [](const ColumnIndex & index, const DB::GlutenParquetColumnIndexFilter::RPNElement & e) { return index.lt(e.value); }); break; - case RPNElement::FUNCTION_GREATER: - CALL_OPERATOR(element, [](const ColumnIndex & index, const RPNElement & e) { return index.gt(e.value); }); + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_GREATER: + CALL_OPERATOR(element, [](const ColumnIndex & index, const DB::GlutenParquetColumnIndexFilter::RPNElement & e) { return index.gt(e.value); }); break; - case RPNElement::FUNCTION_LESS_OR_EQUALS: - CALL_OPERATOR(element, [](const ColumnIndex & index, const RPNElement & e) { return index.ltEg(e.value); }); + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_LESS_OR_EQUALS: + CALL_OPERATOR(element, [](const ColumnIndex & index, const DB::GlutenParquetColumnIndexFilter::RPNElement & e) { return index.ltEg(e.value); }); break; - case RPNElement::FUNCTION_GREATER_OR_EQUALS: - CALL_OPERATOR(element, [](const ColumnIndex & index, const RPNElement & e) { return index.gtEg(e.value); }); + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_GREATER_OR_EQUALS: + CALL_OPERATOR(element, [](const ColumnIndex & index, const DB::GlutenParquetColumnIndexFilter::RPNElement & e) { return index.gtEg(e.value); }); break; - case RPNElement::FUNCTION_IN: - CALL_OPERATOR(element, [](const ColumnIndex & index, const RPNElement & e) { return index.in(e.column); }); + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_IN: + CALL_OPERATOR(element, [](const ColumnIndex & index, const DB::GlutenParquetColumnIndexFilter::RPNElement & e) { return index.in(e.column); }); break; - case RPNElement::FUNCTION_NOT_IN: + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_NOT_IN: rpn_stack.emplace_back(RowRanges::createSingle(rowgroup_count)); break; - case RPNElement::FUNCTION_UNKNOWN: + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_UNKNOWN: rpn_stack.emplace_back(RowRanges::createSingle(rowgroup_count)); break; - case RPNElement::FUNCTION_NOT: + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_NOT: assert(!rpn_stack.empty()); rpn_stack.back() = RowRanges::createSingle(rowgroup_count); break; - case RPNElement::FUNCTION_AND: + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_AND: CALL_LOGICAL_OP(RowRanges::intersection); break; - case RPNElement::FUNCTION_OR: + case DB::GlutenParquetColumnIndexFilter::RPNElement::FUNCTION_OR: CALL_LOGICAL_OP(RowRanges::unionRanges); break; - case RPNElement::ALWAYS_FALSE: - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "RPNElement::ALWAYS_FALSE in ColumnIndexFilter::calculateRowRanges"); + case DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_FALSE: + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_FALSE in ColumnIndexFilter::calculateRowRanges"); break; - case RPNElement::ALWAYS_TRUE: - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "RPNElement::ALWAYS_TRUE in ColumnIndexFilter::calculateRowRanges"); + case DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_TRUE: + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "DB::GlutenParquetColumnIndexFilter::RPNElement::ALWAYS_TRUE in ColumnIndexFilter::calculateRowRanges"); break; } } diff --git a/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.h b/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.h index f5c5cc56168f..aa25d6c08661 100644 --- a/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.h +++ b/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.h @@ -23,6 +23,7 @@ #include #include #include +#include #include namespace DB @@ -159,47 +160,15 @@ using ColumnIndexInt32 = TypedColumnIndex; class ColumnIndexFilter { public: - /// The expression is stored as Reverse Polish Notation. - struct RPNElement - { - enum Function - { - /// Atoms of a Boolean expression. - FUNCTION_EQUALS, - FUNCTION_NOT_EQUALS, - FUNCTION_LESS, - FUNCTION_GREATER, - FUNCTION_LESS_OR_EQUALS, - FUNCTION_GREATER_OR_EQUALS, - FUNCTION_IN, - FUNCTION_NOT_IN, - FUNCTION_UNKNOWN, /// Can take any value. - /// Operators of the logical expression. - FUNCTION_NOT, - FUNCTION_AND, - FUNCTION_OR, - /// Constants - ALWAYS_FALSE, - ALWAYS_TRUE, - }; - - explicit RPNElement(const Function function_ = FUNCTION_UNKNOWN) : function(function_) { } - - Function function = FUNCTION_UNKNOWN; - std::string columnName; - DB::Field value; - DB::ColumnPtr column; - }; - - using RPN = std::vector; - using AtomMap = std::unordered_map; + using RPN = std::vector; + using AtomMap = std::unordered_map; static const AtomMap atom_map; /// Construct key condition from ActionsDAG nodes ColumnIndexFilter(const DB::ActionsDAG & filter_dag, DB::ContextPtr context); private: - static bool extractAtomFromTree(const DB::RPNBuilderTreeNode & node, RPNElement & out); + static bool extractAtomFromTree(const DB::RPNBuilderTreeNode & node, DB::GlutenParquetColumnIndexFilter::RPNElement & out); RPN rpn_; public: diff --git a/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h b/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h index 24dbbc90ade3..5a1e373b3bb8 100644 --- a/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h +++ b/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h @@ -162,8 +162,8 @@ struct ConverterString buf.resize(count); for (size_t i = 0; i < count; ++i) { - StringRef s = column.getDataAt(offset + i); - buf[i] = parquet::ByteArray(static_cast(s.size), reinterpret_cast(s.data)); + auto s = column.getDataAt(offset + i); + buf[i] = parquet::ByteArray(static_cast(s.size()), reinterpret_cast(s.data())); } return buf.data(); } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/Delta/Bitmap/DeltaDVRoaringBitmapArray.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/Delta/Bitmap/DeltaDVRoaringBitmapArray.cpp index 8d37bca0546b..d33fe2107488 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/Delta/Bitmap/DeltaDVRoaringBitmapArray.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/Delta/Bitmap/DeltaDVRoaringBitmapArray.cpp @@ -22,7 +22,7 @@ #include #include #include -#include +#include #include #include diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/Delta/Bitmap/DeltaDVRoaringBitmapArray.h b/cpp-ch/local-engine/Storages/SubstraitSource/Delta/Bitmap/DeltaDVRoaringBitmapArray.h index a2d80080e1d0..8cd0fecbdfc1 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/Delta/Bitmap/DeltaDVRoaringBitmapArray.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/Delta/Bitmap/DeltaDVRoaringBitmapArray.h @@ -20,7 +20,7 @@ #include #include #include -#include +#include namespace local_engine { diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/Delta/DeltaWriter.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/Delta/DeltaWriter.cpp index a5a1a887bfcb..47c6de618850 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/Delta/DeltaWriter.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/Delta/DeltaWriter.cpp @@ -159,7 +159,8 @@ void DeltaWriter::writeDeletionVector(const DB::Block & block) for (size_t row_idx = 0; row_idx < block.rows(); row_idx++) { const auto file_path = file_path_columns.column->getDataAt(row_idx); - auto bitmap = bitmap_columns.column->getDataAt(row_idx).toString(); + const auto str_bitmap = bitmap_columns.column->getDataAt(row_idx); + auto bitmap = std::string(str_bitmap.data(), str_bitmap.size()); auto cardinality = cardinality_src_columns.column->get64(row_idx); // alisa deletedRowIndexCount if (size_of_current_bin > 0 && bitmap.length() + size_of_current_bin > packing_target_size) @@ -189,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.data()); dv_descriptor_column->insert(dv_descriptor_field); matched_row_count_col->insert(cardinality); continue; @@ -213,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.data()); dv_descriptor_column->insert(dv_descriptor_field); matched_row_count_col->insert(cardinality); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/Iceberg/SimpleParquetReader.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/Iceberg/SimpleParquetReader.cpp index 660a38f8bb92..5250f2aecb47 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/Iceberg/SimpleParquetReader.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/Iceberg/SimpleParquetReader.cpp @@ -63,7 +63,7 @@ SimpleParquetReader::SimpleParquetReader( // TODO: set min_bytes_for_seek ParquetReader::Settings settings{ .arrow_properties = parquet::ArrowReaderProperties(), - .reader_properties = parquet::ReaderProperties(ArrowMemoryPool::instance()), + .reader_properties = parquet::ReaderProperties(arrow::default_memory_pool()), .format_settings = format_settings}; read_buffer_reader_ = read_buffer_builder->build(file_info); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp index 854bd5a3ddba..116025ce233e 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp @@ -71,8 +71,8 @@ ORCFormatFile::createInputFormat(const DB::Block & header, const std::shared_ptr const String mapped_timezone = DateTimeUtil::convertTimeZone(config_timezone); format_settings.orc.reader_time_zone_name = mapped_timezone; } - //TODO: support prefetch - auto parser_group = std::make_shared(filter_actions_dag, context, nullptr); + //TODO: rebase-25.12, support prefetch and how to pass row_level_filter, prewhere_info + auto parser_group = std::make_shared(filter_actions_dag, context, nullptr, nullptr, nullptr); auto input_format = std::make_shared(*read_buffer, toShared(header), format_settings, false, 0, parser_group); return std::make_shared(std::move(read_buffer), input_format); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp index f8e3a3c91418..6eb4951a9b0b 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp @@ -26,10 +26,12 @@ #include #include #include +#include #include #include #include #include +#include namespace DB { @@ -105,7 +107,7 @@ class ParquetInputFormat : public FormatFile::InputFormat { assert(outputHeader.columns()); assert(row_index_reader); - // TODO: format_settings_.parquet.max_block_size + // TODO: rebase-25.12, format_settings_.parquet.max_block_size Columns cols{row_index_reader->readBatch(8192)}; size_t rows = cols[0]->size(); return Chunk(std::move(cols), rows); @@ -149,7 +151,8 @@ ParquetFormatFile::createInputFormat(const Block & header, const std::shared_ptr assert(read_buffer_); bool readRowIndex = hasMetaColumns(header); - bool usePageIndexReader = (use_pageindex_reader || readRowIndex) && onlyHasFlatType(header); + bool onlyFlatType = onlyHasFlatType(header); + bool usePageIndexReader = (use_pageindex_reader || readRowIndex) && onlyFlatType; auto format_settings = getFormatSettings(context); auto read_header = toShared(DeltaVirtualMeta::removeMetaColumns(removeMetaColumns(header))); @@ -205,10 +208,27 @@ ParquetFormatFile::createInputFormat(const Block & header, const std::shared_ptr // We need to disable fiter push down and read all row groups, so that we can get correct row index. format_settings.parquet.filter_push_down = false; } - auto parser_group = std::make_shared(filter_actions_dag, context, nullptr); + // TODO: rebase-25.12, support prefetch and how to pass row_level_filter, prewhere_info + auto parser_group = std::make_shared(filter_actions_dag, context, nullptr, nullptr, nullptr); auto parser_shared_resources = std::make_shared(context->getSettingsRef(), /*num_streams_=*/1); - auto input = std::make_shared(*read_buffer_, read_header, format_settings, parser_shared_resources, parser_group, 8192); - return std::make_shared(std::move(read_buffer_), input, std::move(provider), *read_header, header); + + size_t min_bytes_for_seek = format_settings.parquet.local_read_min_bytes_for_seek; + if (format_settings.parquet.use_native_reader_v3 && !readRowIndex && onlyFlatType) + { + LOG_TRACE( + &Poco::Logger::get("ParquetFormatFile"), + "Using native parquet reader v3"); + auto input = std::make_shared(*read_buffer_, read_header, format_settings, parser_shared_resources, parser_group, min_bytes_for_seek); + return std::make_shared(std::move(read_buffer_), input, std::move(provider), *read_header, header); + } + else + { + LOG_TRACE( + &Poco::Logger::get("ParquetFormatFile"), + "Using native parquet reader"); + auto input = std::make_shared(*read_buffer_, read_header, format_settings, parser_shared_resources, parser_group, min_bytes_for_seek); + return std::make_shared(std::move(read_buffer_), input, std::move(provider), *read_header, header); + } }; return usePageIndexReader ? createVectorizedFormat() : createParquetBlockInputFormat(); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp index 258a1b89d976..6d9996570988 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp @@ -52,9 +52,10 @@ #include #include #include +#include #if USE_AZURE_BLOB_STORAGE -#include +#include #endif #if USE_AWS_S3 @@ -560,8 +561,8 @@ class S3FileReadBufferBuilder : public ReadBufferBuilder context->getSettingsRef()[Setting::s3_slow_all_threads_after_retryable_error], context->getSettingsRef()[DB::Setting::enable_s3_requests_logging], false, - nullptr, - nullptr); + {}, + {}); client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", 10000); client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 5000); @@ -912,7 +913,7 @@ std::unique_ptr ReadBufferBuilder::wrapWithParallelIfNeeded( return wrapInParallelReadBufferIfSupported( {std::move(in)}, - DB::threadPoolCallbackRunnerUnsafe(DB::getIOThreadPool().get(), "ParallelRead"), + DB::threadPoolCallbackRunnerUnsafe(DB::getIOThreadPool().get(), ThreadName::PARALLEL_READ), max_download_threads, max_download_buffer_size, file_size); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h index 038e864b8e77..f3b17f0105dd 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h @@ -18,7 +18,7 @@ #include #include -#include +#include #include #include #include diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 0f56336a688c..b35f8b33d120 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -404,7 +404,7 @@ JNIEXPORT jbyte Java_org_apache_gluten_vectorized_CHColumnVector_nativeGetByte( DB::ColumnPtr nested_col = col.column; if (const auto * nullable_col = checkAndGetColumn(nested_col.get())) nested_col = nullable_col->getNestedColumnPtr(); - return reinterpret_cast(nested_col->getDataAt(row_id).data)[0]; + return reinterpret_cast(nested_col->getDataAt(row_id).data())[0]; LOCAL_ENGINE_JNI_METHOD_END(env, 0) } @@ -416,7 +416,7 @@ JNIEXPORT jshort Java_org_apache_gluten_vectorized_CHColumnVector_nativeGetShort DB::ColumnPtr nested_col = col.column; if (const auto * nullable_col = checkAndGetColumn(nested_col.get())) nested_col = nullable_col->getNestedColumnPtr(); - return reinterpret_cast(nested_col->getDataAt(row_id).data)[0]; + return reinterpret_cast(nested_col->getDataAt(row_id).data())[0]; LOCAL_ENGINE_JNI_METHOD_END(env, -1) } @@ -481,7 +481,7 @@ JNIEXPORT jstring Java_org_apache_gluten_vectorized_CHColumnVector_nativeGetStri nested_col = nullable_col->getNestedColumnPtr(); const auto * string_col = checkAndGetColumn(nested_col.get()); auto result = string_col->getDataAt(row_id); - return local_engine::charTojstring(env, result.toString().c_str()); + return local_engine::charTojstring(env, result.data()); LOCAL_ENGINE_JNI_METHOD_END(env, local_engine::charTojstring(env, "")) } diff --git a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala index 93c3fe631f6d..cc5fd3cc5fd9 100644 --- a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala +++ b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala @@ -494,12 +494,14 @@ object GlutenQueryTest extends Assertions { df: DataFrame, expectedAnswer: Seq[Row], checkToRDD: Boolean = true): Option[String] = { - if (checkToRDD) { + val execId = if (checkToRDD) { val executionId = getNextExecutionId SQLExecution.withExecutionId(df.sparkSession, executionId) { df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] } - BackendsApiManager.getTransformerApiInstance.invalidateSQLExecutionResource(executionId) + executionId + } else { + "" } val sparkAnswer = @@ -518,7 +520,7 @@ object GlutenQueryTest extends Assertions { } val sortedColIdxes = getOuterSortedColIdxes(df) - sameRows(expectedAnswer, sparkAnswer, sortedColIdxes, df.schema.length).map { + val res = sameRows(expectedAnswer, sparkAnswer, sortedColIdxes, df.schema.length).map { results => s""" |Results do not match for query: @@ -530,6 +532,12 @@ object GlutenQueryTest extends Assertions { |$results """.stripMargin } + + if (execId.nonEmpty) { + BackendsApiManager.getTransformerApiInstance.invalidateSQLExecutionResource(execId) + } + + res } def getNextExecutionId: String = { From 29dce5d7784fc57b421e71ec9c2e845e0b718f7b Mon Sep 17 00:00:00 2001 From: Zhichao Zhang Date: Wed, 11 Mar 2026 10:12:51 +0800 Subject: [PATCH 2/3] fix --- .../org/apache/spark/sql/GlutenQueryTest.scala | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala index cc5fd3cc5fd9..93c3fe631f6d 100644 --- a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala +++ b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala @@ -494,14 +494,12 @@ object GlutenQueryTest extends Assertions { df: DataFrame, expectedAnswer: Seq[Row], checkToRDD: Boolean = true): Option[String] = { - val execId = if (checkToRDD) { + if (checkToRDD) { val executionId = getNextExecutionId SQLExecution.withExecutionId(df.sparkSession, executionId) { df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] } - executionId - } else { - "" + BackendsApiManager.getTransformerApiInstance.invalidateSQLExecutionResource(executionId) } val sparkAnswer = @@ -520,7 +518,7 @@ object GlutenQueryTest extends Assertions { } val sortedColIdxes = getOuterSortedColIdxes(df) - val res = sameRows(expectedAnswer, sparkAnswer, sortedColIdxes, df.schema.length).map { + sameRows(expectedAnswer, sparkAnswer, sortedColIdxes, df.schema.length).map { results => s""" |Results do not match for query: @@ -532,12 +530,6 @@ object GlutenQueryTest extends Assertions { |$results """.stripMargin } - - if (execId.nonEmpty) { - BackendsApiManager.getTransformerApiInstance.invalidateSQLExecutionResource(execId) - } - - res } def getNextExecutionId: String = { From 45a83386c3f45ab4190b9a7c38339d2394f2dfc6 Mon Sep 17 00:00:00 2001 From: Zhichao Zhang Date: Wed, 11 Mar 2026 15:58:42 +0800 Subject: [PATCH 3/3] revert --- cpp-ch/clickhouse.version | 2 +- .../apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala | 2 ++ .../apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala | 2 ++ .../apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala | 2 ++ .../apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala | 2 ++ .../apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala | 2 ++ .../apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala | 2 ++ 7 files changed, 13 insertions(+), 1 deletion(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index ac8ea9d696e1..55fe9671ca06 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence CH_BRANCH=rebase_ch/20260310_25_12 -CH_COMMIT=b7393160b95 +CH_COMMIT=408fb65018b diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index dc44facd7ef9..76dae81a5d40 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -1549,6 +1549,8 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("Legacy mode - nested map with struct as key type") .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") .exclude("SPARK-35640: int as long should throw schema incompatible error") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader true") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader false") enableSuite[GlutenParquetInteroperabilitySuite].exclude("parquet timestamp conversion") enableSuite[GlutenParquetProtobufCompatibilitySuite].exclude("struct with unannotated array") enableSuite[GlutenParquetRebaseDatetimeV1Suite] diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index c6bb67748e9b..b4d0856e1d9e 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -1389,6 +1389,8 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") .exclude("SPARK-35640: int as long should throw schema incompatible error") .exclude("SPARK-36726: test incorrect Parquet row group file offset") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader true") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader false") enableSuite[GlutenParquetInteroperabilitySuite].exclude("parquet timestamp conversion") enableSuite[GlutenParquetProtobufCompatibilitySuite].exclude("struct with unannotated array") enableSuite[GlutenParquetRebaseDatetimeV1Suite] diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 71b621081746..df3eb5fd7f52 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -1258,6 +1258,8 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") .exclude("SPARK-35640: int as long should throw schema incompatible error") .exclude("SPARK-36726: test incorrect Parquet row group file offset") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader true") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader false") enableSuite[GlutenParquetInteroperabilitySuite].exclude("parquet timestamp conversion") enableSuite[GlutenParquetProtobufCompatibilitySuite].exclude("struct with unannotated array") enableSuite[GlutenParquetRebaseDatetimeV1Suite] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 29d7534e8fae..57b2b1ca95de 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -1775,6 +1775,8 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35640: int as long should throw schema incompatible error") // Velox parquet reader not allow offset zero. .includeCH("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader true") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader false") enableSuite[GlutenParquetInteroperabilitySuite] .exclude("parquet timestamp conversion") enableSuite[GlutenParquetProtobufCompatibilitySuite] diff --git a/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index ec99089c324e..7ff5f04bd13b 100644 --- a/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -1751,6 +1751,8 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35640: int as long should throw schema incompatible error") // Velox parquet reader not allow offset zero. .includeCH("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader true") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader false") enableSuite[GlutenParquetInteroperabilitySuite] .exclude("parquet timestamp conversion") enableSuite[GlutenParquetProtobufCompatibilitySuite] diff --git a/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index ec99089c324e..7ff5f04bd13b 100644 --- a/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -1751,6 +1751,8 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35640: int as long should throw schema incompatible error") // Velox parquet reader not allow offset zero. .includeCH("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader true") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader false") enableSuite[GlutenParquetInteroperabilitySuite] .exclude("parquet timestamp conversion") enableSuite[GlutenParquetProtobufCompatibilitySuite]