From 4e30ed1a09a833c468b3275d23f329c8ba158c4d Mon Sep 17 00:00:00 2001 From: Chang chen Date: Thu, 19 Sep 2024 22:23:19 +0800 Subject: [PATCH] [GLUTEN-7028][CH][Part-2] Refactor: Move MergeTree related UT to mergetree module (#7279) * Add CHConf * Move MergeTree related UT to mergetree module * fix scala stye * spark32 spark33 spark35 * More CH Conf * update per https://github.com/apache/incubator-gluten/pull/7265 - Use CHConf - use CHConf.prefixOf() instead of "spark.gluten.sql.columnar.backend.ch." - settingsKey => runtimeSettings - configKey => runtimeConfig - CH => CONF_PREFIX * fix due to https://github.com/apache/incubator-gluten/pull/7263 --- .../ClickhouseOptimisticTransaction.scala | 4 +- .../ClickhouseOptimisticTransaction.scala | 4 +- .../io/delta/tables/ClickhouseTable.scala | 2 +- .../ClickhouseOptimisticTransaction.scala | 4 +- .../backendsapi/clickhouse/CHBackend.scala | 48 ++-- .../backendsapi/clickhouse/CHConf.scala | 55 +++++ .../clickhouse/CHListenerApi.scala | 14 +- .../clickhouse/CHTransformerApi.scala | 27 +-- .../expression/CHExpressionTransformer.scala | 16 +- .../commands/GlutenCacheFilesCommand.scala | 6 +- .../utils/MergeTreePartsPartitionsUtil.scala | 14 +- .../v2/clickhouse/ClickHouseConfig.scala | 24 +- .../scala/org/apache/gluten/RunTPCHTest.scala | 10 +- ...kHouseColumnarMemorySortShuffleSuite.scala | 6 +- ...tenClickHouseColumnarShuffleAQESuite.scala | 15 +- ...utenClickHouseDeltaParquetWriteSuite.scala | 215 +++++++++--------- .../GlutenClickHouseFileFormatSuite.scala | 39 +--- .../execution/GlutenClickHouseJoinSuite.scala | 6 +- .../GlutenClickHouseS3SourceSuite.scala | 25 +- .../GlutenClickHouseSyntheticDataSuite.scala | 9 +- .../GlutenClickHouseTPCDSAbstractSuite.scala | 3 +- .../GlutenClickHouseTPCHAbstractSuite.scala | 5 +- .../GlutenClickHouseTPCHBucketSuite.scala | 175 +++++++------- ...nClickHouseTPCHNotNullSkipIndexSuite.scala | 11 +- ...ClickHouseTPCHNullableSkipIndexSuite.scala | 11 +- ...ClickHouseWholeStageTransformerSuite.scala | 153 ++++--------- .../GlutenFunctionValidateSuite.scala | 29 +-- .../GlutenClickhouseFunctionSuite.scala | 3 +- .../hive/GlutenClickHouseHiveTableSuite.scala | 13 +- ...lutenClickHouseNativeWriteTableSuite.scala | 7 +- .../GlutenClickHouseTableAfterRestart.scala | 20 +- ...tenClickHouseMergeTreeCacheDataSuite.scala | 21 +- ...utenClickHouseMergeTreeOptimizeSuite.scala | 66 +++--- ...ickHouseMergeTreePathBasedWriteSuite.scala | 62 ++--- ...nClickHouseMergeTreeWriteOnHDFSSuite.scala | 29 ++- ...eTreeWriteOnHDFSWithRocksDBMetaSuite.scala | 29 ++- ...tenClickHouseMergeTreeWriteOnS3Suite.scala | 32 ++- .../GlutenClickHouseMergeTreeWriteSuite.scala | 136 ++++++----- ...rgeTreeWriteTaskNotSerializableSuite.scala | 12 +- ...ClickhouseMergetreeSoftAffinitySuite.scala | 3 +- .../GlutenClickHouseTPCHMetricsSuite.scala | 12 +- .../GlutenParquetColumnIndexSuite.scala | 7 +- ...kHouseTPCDSParquetGraceHashJoinSuite.scala | 6 +- .../tpch/GlutenClickHouseHDFSSuite.scala | 20 +- ...seTPCHColumnarShuffleParquetAQESuite.scala | 6 +- .../GlutenClickHouseTPCHParquetAQESuite.scala | 4 +- ...utenClickHouseTPCHParquetBucketSuite.scala | 18 +- .../GlutenClickHouseTPCHParquetRFSuite.scala | 2 +- ...enClickHouseTPCHSaltNullParquetSuite.scala | 30 ++- .../benchmarks/CHOptimizeRuleBenchmark.scala | 4 +- 50 files changed, 711 insertions(+), 761 deletions(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => mergetree}/GlutenClickHouseMergeTreeCacheDataSuite.scala (97%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => mergetree}/GlutenClickHouseMergeTreeOptimizeSuite.scala (92%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => mergetree}/GlutenClickHouseMergeTreePathBasedWriteSuite.scala (96%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => mergetree}/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala (96%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => mergetree}/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala (96%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => mergetree}/GlutenClickHouseMergeTreeWriteOnS3Suite.scala (96%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => mergetree}/GlutenClickHouseMergeTreeWriteSuite.scala (95%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => mergetree}/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala (93%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => mergetree}/GlutenClickhouseMergetreeSoftAffinitySuite.scala (96%) diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 9575a654be92..773cd35e9367 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.delta -import org.apache.gluten.backendsapi.clickhouse.CHBackend +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution.ColumnarToRowExecBase import org.apache.spark.SparkException @@ -128,7 +128,7 @@ class ClickhouseOptimisticTransaction( spark.conf.getAll.foreach( entry => { if ( - entry._1.startsWith(s"${CHBackend.CONF_PREFIX}.runtime_settings") + CHConf.startWithSettings(entry._1) || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key) ) { options += (entry._1 -> entry._2) diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 9575a654be92..773cd35e9367 100644 --- a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.delta -import org.apache.gluten.backendsapi.clickhouse.CHBackend +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution.ColumnarToRowExecBase import org.apache.spark.SparkException @@ -128,7 +128,7 @@ class ClickhouseOptimisticTransaction( spark.conf.getAll.foreach( entry => { if ( - entry._1.startsWith(s"${CHBackend.CONF_PREFIX}.runtime_settings") + CHConf.startWithSettings(entry._1) || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key) ) { options += (entry._1 -> entry._2) diff --git a/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala b/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala index 790b4c1f8a37..e747c87c6a67 100644 --- a/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala +++ b/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala @@ -91,7 +91,7 @@ object ClickhouseTable { val badOptions = hadoopConf.filterKeys { k => !DeltaTableUtils.validDeltaTableHadoopPrefixes.exists(k.startsWith) }.toMap - if (!badOptions.isEmpty) { + if (badOptions.nonEmpty) { throw DeltaErrors.unsupportedDeltaTableForPathHadoopConf(badOptions) } val fileSystemOptions: Map[String, String] = hadoopConf.toMap diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 6db39b72781f..00940a4851e0 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.delta -import org.apache.gluten.backendsapi.clickhouse.CHBackend +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution.ColumnarToRowExecBase import org.apache.spark.SparkException @@ -140,7 +140,7 @@ class ClickhouseOptimisticTransaction( spark.conf.getAll.foreach( entry => { if ( - entry._1.startsWith(s"${CHBackend.CONF_PREFIX}.runtime_settings") + CHConf.startWithSettings(entry._1) || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key) ) { options += (entry._1 -> entry._2) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 0ff4010a33ec..cecdd7477a3a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -62,8 +62,7 @@ class CHBackend extends SubstraitBackend { } object CHBackend { - val BACKEND_NAME = "ch" - val CONF_PREFIX: String = GlutenConfig.prefixOf(BACKEND_NAME) + val BACKEND_NAME: String = CHConf.BACKEND_NAME } object CHBackendSettings extends BackendSettingsApi with Logging { @@ -74,13 +73,11 @@ object CHBackendSettings extends BackendSettingsApi with Logging { // experimental: when the files count per partition exceeds this threshold, // it will put the files into one partition. val GLUTEN_CLICKHOUSE_FILES_PER_PARTITION_THRESHOLD: String = - GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + - ".files.per.partition.threshold" + CHConf.prefixOf("files.per.partition.threshold") val GLUTEN_CLICKHOUSE_FILES_PER_PARTITION_THRESHOLD_DEFAULT = "-1" private val GLUTEN_CLICKHOUSE_CUSTOMIZED_SHUFFLE_CODEC_ENABLE: String = - GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + - ".customized.shuffle.codec.enable" + CHConf.prefixOf("customized.shuffle.codec.enable") private val GLUTEN_CLICKHOUSE_CUSTOMIZED_SHUFFLE_CODEC_ENABLE_DEFAULT = false lazy val useCustomizedShuffleCodec: Boolean = SparkEnv.get.conf.getBoolean( CHBackendSettings.GLUTEN_CLICKHOUSE_CUSTOMIZED_SHUFFLE_CODEC_ENABLE, @@ -88,8 +85,7 @@ object CHBackendSettings extends BackendSettingsApi with Logging { ) private val GLUTEN_CLICKHOUSE_CUSTOMIZED_BUFFER_SIZE: String = - GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + - ".customized.buffer.size" + CHConf.prefixOf("customized.buffer.size") private val GLUTEN_CLICKHOUSE_CUSTOMIZED_BUFFER_SIZE_DEFAULT = 4096 lazy val customizeBufferSize: Int = SparkEnv.get.conf.getInt( CHBackendSettings.GLUTEN_CLICKHOUSE_CUSTOMIZED_BUFFER_SIZE, @@ -97,8 +93,7 @@ object CHBackendSettings extends BackendSettingsApi with Logging { ) val GLUTEN_CLICKHOUSE_BROADCAST_CACHE_EXPIRED_TIME: String = - GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + - ".broadcast.cache.expired.time" + CHConf.prefixOf("broadcast.cache.expired.time") // unit: SECONDS, default 1 day val GLUTEN_CLICKHOUSE_BROADCAST_CACHE_EXPIRED_TIME_DEFAULT: Int = 86400 @@ -106,8 +101,7 @@ object CHBackendSettings extends BackendSettingsApi with Logging { // The algorithm for hash partition of the shuffle private val GLUTEN_CLICKHOUSE_SHUFFLE_HASH_ALGORITHM: String = - GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + - ".shuffle.hash.algorithm" + CHConf.prefixOf("shuffle.hash.algorithm") // valid values are: cityHash64 or sparkMurmurHash3_32 private val GLUTEN_CLICKHOUSE_SHUFFLE_HASH_ALGORITHM_DEFAULT = "sparkMurmurHash3_32" def shuffleHashAlgorithm: String = { @@ -122,25 +116,19 @@ object CHBackendSettings extends BackendSettingsApi with Logging { } } - val GLUTEN_CLICKHOUSE_AFFINITY_MODE: String = - GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + ".affinity.mode" + private val GLUTEN_CLICKHOUSE_AFFINITY_MODE: String = CHConf.prefixOf("affinity.mode") val SOFT: String = "soft" val FORCE: String = "force" private val GLUTEN_CLICKHOUSE_AFFINITY_MODE_DEFAULT = SOFT - val GLUTEN_MAX_BLOCK_SIZE: String = - GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + - ".runtime_settings.max_block_size" + private val GLUTEN_MAX_BLOCK_SIZE: String = CHConf.runtimeSettings("max_block_size") // Same as default value in clickhouse - val GLUTEN_MAX_BLOCK_SIZE_DEFAULT = 65409 - val GLUTEN_MAX_SHUFFLE_READ_BYTES: String = - GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + - ".runtime_config.max_source_concatenate_bytes" - val GLUTEN_MAX_SHUFFLE_READ_BYTES_DEFAULT = GLUTEN_MAX_BLOCK_SIZE_DEFAULT * 256 + private val GLUTEN_MAX_BLOCK_SIZE_DEFAULT = 65409 + private val GLUTEN_MAX_SHUFFLE_READ_BYTES: String = + CHConf.runtimeConfig("max_source_concatenate_bytes") + private val GLUTEN_MAX_SHUFFLE_READ_BYTES_DEFAULT = GLUTEN_MAX_BLOCK_SIZE_DEFAULT * 256 - val GLUTEN_AQE_PROPAGATEEMPTY: String = - GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + - ".aqe.propagate.empty.relation" + val GLUTEN_AQE_PROPAGATEEMPTY: String = CHConf.prefixOf("aqe.propagate.empty.relation") def affinityMode: String = { SparkEnv.get.conf @@ -368,15 +356,15 @@ object CHBackendSettings extends BackendSettingsApi with Logging { // Need to enable AQE def enableReorderHashJoinTables(): Boolean = { SparkEnv.get.conf.getBoolean( - "spark.gluten.sql.columnar.backend.ch.enable_reorder_hash_join_tables", - true + CHConf.prefixOf("enable_reorder_hash_join_tables"), + defaultValue = true ) } // The threshold to reorder hash join tables, if The result of dividing two tables' size is // large then this threshold, reorder the tables. e.g. a/b > threshold or b/a > threshold def reorderHashJoinTablesThreshold(): Int = { SparkEnv.get.conf.getInt( - "spark.gluten.sql.columnar.backend.ch.reorder_hash_join_tables_thresdhold", + CHConf.prefixOf("reorder_hash_join_tables_thresdhold"), 10 ) } @@ -385,8 +373,8 @@ object CHBackendSettings extends BackendSettingsApi with Logging { // for example, select a, b, sum(c+d) from t group by a, b with cube def enablePushdownPreProjectionAheadExpand(): Boolean = { SparkEnv.get.conf.getBoolean( - "spark.gluten.sql.columnar.backend.ch.enable_pushdown_preprojection_ahead_expand", - true + CHConf.prefixOf("enable_pushdown_preprojection_ahead_expand"), + defaultValue = true ) } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala new file mode 100644 index 000000000000..16201dbad120 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.backendsapi.clickhouse + +import org.apache.gluten.GlutenConfig + +import org.apache.spark.SparkConf + +object CHConf { + private[clickhouse] val BACKEND_NAME: String = "ch" + private[clickhouse] val CONF_PREFIX: String = GlutenConfig.prefixOf(BACKEND_NAME) + private val RUNTIME_SETTINGS: String = s"$CONF_PREFIX.runtime_settings" + private val RUNTIME_CONFIG = s"$CONF_PREFIX.runtime_config" + implicit class GlutenCHConf(conf: SparkConf) { + def setCHSettings(settings: (String, String)*): SparkConf = { + settings.foreach { case (k, v) => conf.set(runtimeSettings(k), v) } + conf + } + + def setCHSettings[T](k: String, v: T): SparkConf = { + conf.set(runtimeSettings(k), v.toString) + conf + } + + def setCHConfig(config: (String, String)*): SparkConf = { + config.foreach { case (k, v) => conf.set(runtimeConfig(k), v) } + conf + } + + def setCHConfig[T](k: String, v: T): SparkConf = { + conf.set(runtimeConfig(k), v.toString) + conf + } + } + + def prefixOf(key: String): String = s"$CONF_PREFIX.$key" + def runtimeConfig(key: String): String = s"$RUNTIME_CONFIG.$key" + def runtimeSettings(key: String): String = s"$RUNTIME_SETTINGS.$key" + + def startWithSettings(key: String): Boolean = key.startsWith(RUNTIME_SETTINGS) +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala index 9ff30d82861c..58acda88fba5 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala @@ -81,17 +81,13 @@ class CHListenerApi extends ListenerApi with Logging { JniLibLoader.loadFromPath(executorLibPath, true) } // Add configs - conf.set( - s"${CHBackend.CONF_PREFIX}.runtime_config.timezone", - conf.get("spark.sql.session.timeZone", TimeZone.getDefault.getID)) - conf.set( - s"${CHBackend.CONF_PREFIX}.runtime_config" + - s".local_engine.settings.log_processors_profiles", - "true") + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + conf.setCHConfig( + "timezone" -> conf.get("spark.sql.session.timeZone", TimeZone.getDefault.getID), + "local_engine.settings.log_processors_profiles" -> "true") // add memory limit for external sort - val externalSortKey = s"${CHBackend.CONF_PREFIX}.runtime_settings" + - s".max_bytes_before_external_sort" + val externalSortKey = CHConf.runtimeSettings("max_bytes_before_external_sort") if (conf.getLong(externalSortKey, -1) < 0) { if (conf.getBoolean("spark.memory.offHeap.enabled", defaultValue = false)) { val memSize = JavaUtils.byteStringAsBytes(conf.get("spark.memory.offHeap.size")) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala index 9653256256bd..62fe3d9fa97a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala @@ -87,32 +87,33 @@ class CHTransformerApi extends TransformerApi with Logging { override def postProcessNativeConfig( nativeConfMap: util.Map[String, String], backendPrefix: String): Unit = { - val settingPrefix = backendPrefix + ".runtime_settings." + + require(backendPrefix == CHConf.CONF_PREFIX) if (nativeConfMap.getOrDefault("spark.memory.offHeap.enabled", "false").toBoolean) { val offHeapSize = nativeConfMap.getOrDefault("spark.gluten.memory.offHeap.size.in.bytes", "0").toLong if (offHeapSize > 0) { // Only set default max_bytes_before_external_group_by for CH when it is not set explicitly. - val groupBySpillKey = settingPrefix + "max_bytes_before_external_group_by"; + val groupBySpillKey = CHConf.runtimeSettings("max_bytes_before_external_group_by") if (!nativeConfMap.containsKey(groupBySpillKey)) { val groupBySpillValue = offHeapSize * 0.5 nativeConfMap.put(groupBySpillKey, groupBySpillValue.toLong.toString) } - val maxMemoryUsageKey = settingPrefix + "max_memory_usage"; + val maxMemoryUsageKey = CHConf.runtimeSettings("max_memory_usage") if (!nativeConfMap.containsKey(maxMemoryUsageKey)) { val maxMemoryUsageValue = offHeapSize - nativeConfMap.put(maxMemoryUsageKey, maxMemoryUsageValue.toLong.toString) + nativeConfMap.put(maxMemoryUsageKey, maxMemoryUsageValue.toString) } // Only set default max_bytes_before_external_join for CH when join_algorithm is grace_hash - val joinAlgorithmKey = settingPrefix + "join_algorithm"; + val joinAlgorithmKey = CHConf.runtimeSettings("join_algorithm") if ( nativeConfMap.containsKey(joinAlgorithmKey) && nativeConfMap.get(joinAlgorithmKey) == "grace_hash" ) { - val joinSpillKey = settingPrefix + "max_bytes_in_join"; + val joinSpillKey = CHConf.runtimeSettings("max_bytes_in_join") if (!nativeConfMap.containsKey(joinSpillKey)) { val joinSpillValue = offHeapSize * 0.7 nativeConfMap.put(joinSpillKey, joinSpillValue.toLong.toString) @@ -127,24 +128,24 @@ class CHTransformerApi extends TransformerApi with Logging { } } - val hdfsConfigPrefix = backendPrefix + ".runtime_config.hdfs." - injectConfig("spark.hadoop.input.connect.timeout", hdfsConfigPrefix + "input_connect_timeout") - injectConfig("spark.hadoop.input.read.timeout", hdfsConfigPrefix + "input_read_timeout") - injectConfig("spark.hadoop.input.write.timeout", hdfsConfigPrefix + "input_write_timeout") + val hdfsConfigPrefix = CHConf.runtimeConfig("hdfs") + injectConfig("spark.hadoop.input.connect.timeout", s"$hdfsConfigPrefix.input_connect_timeout") + injectConfig("spark.hadoop.input.read.timeout", s"$hdfsConfigPrefix.input_read_timeout") + injectConfig("spark.hadoop.input.write.timeout", s"$hdfsConfigPrefix.input_write_timeout") injectConfig( "spark.hadoop.dfs.client.log.severity", - hdfsConfigPrefix + "dfs_client_log_severity") + s"$hdfsConfigPrefix.dfs_client_log_severity") // TODO: set default to true when metrics could be collected // while ch query plan optimization is enabled. - val planOptKey = settingPrefix + "query_plan_enable_optimizations" + val planOptKey = CHConf.runtimeSettings("query_plan_enable_optimizations") if (!nativeConfMap.containsKey(planOptKey)) { nativeConfMap.put(planOptKey, "false") } // Respect spark config spark.sql.orc.compression.codec for CH backend // TODO: consider compression or orc.compression in table options. - val orcCompressionKey = settingPrefix + "output_format_orc_compression_method" + val orcCompressionKey = CHConf.runtimeSettings("output_format_orc_compression_method") if (!nativeConfMap.containsKey(orcCompressionKey)) { if (nativeConfMap.containsKey("spark.sql.orc.compression.codec")) { val compression = nativeConfMap.get("spark.sql.orc.compression.codec").toLowerCase() diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/expression/CHExpressionTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/expression/CHExpressionTransformer.scala index f430b1141cf9..fe8d23f9a958 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/expression/CHExpressionTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/expression/CHExpressionTransformer.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.expression -import org.apache.gluten.backendsapi.clickhouse.CHBackend +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.expression.ConverterUtils.FunctionConfig import org.apache.gluten.substrait.expression._ @@ -70,7 +70,7 @@ case class CHTruncTimestampTransformer( if ( timeZoneIgnore && timeZoneId.nonEmpty && !timeZoneId.get.equalsIgnoreCase( - SQLConf.get.getConfString(s"${CHBackend.CONF_PREFIX}.runtime_config.timezone") + SQLConf.get.getConfString(s"${CHConf.runtimeConfig("timezone")}") ) ) { throw new GlutenNotSupportException( @@ -157,23 +157,23 @@ case class CHPosExplodeTransformer( // Output (pos, col) when input is array type val structType = StructType( Array( - StructField("pos", IntegerType, false), + StructField("pos", IntegerType, nullable = false), StructField("col", a.elementType, a.containsNull))) ExpressionBuilder.makeScalarFunction( funcId, Lists.newArrayList(childNode), - ConverterUtils.getTypeNode(structType, false)) + ConverterUtils.getTypeNode(structType, nullable = false)) case m: MapType => // Output (pos, key, value) when input is map type val structType = StructType( Array( - StructField("pos", IntegerType, false), - StructField("key", m.keyType, false), + StructField("pos", IntegerType, nullable = false), + StructField("key", m.keyType, nullable = false), StructField("value", m.valueType, m.valueContainsNull))) ExpressionBuilder.makeScalarFunction( funcId, Lists.newArrayList(childNode), - ConverterUtils.getTypeNode(structType, false)) + ConverterUtils.getTypeNode(structType, nullable = false)) case _ => throw new GlutenNotSupportException(s"posexplode($childType) not supported yet.") } @@ -225,7 +225,7 @@ case class GetArrayItemTransformer( Seq(IntegerType, getArrayItem.right.dataType), FunctionConfig.OPT) val addFunctionId = ExpressionBuilder.newScalarFunction(functionMap, addFunctionName) - val literalNode = ExpressionBuilder.makeLiteral(1.toInt, IntegerType, false) + val literalNode = ExpressionBuilder.makeLiteral(1, IntegerType, false) rightNode = ExpressionBuilder.makeScalarFunction( addFunctionId, Lists.newArrayList(literalNode, rightNode), diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/commands/GlutenCacheFilesCommand.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/commands/GlutenCacheFilesCommand.scala index b2640df789e9..aec414f05596 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/commands/GlutenCacheFilesCommand.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/commands/GlutenCacheFilesCommand.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.sql.execution.commands -import org.apache.gluten.GlutenConfig -import org.apache.gluten.backendsapi.clickhouse.CHBackend +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.substrait.rel.LocalFilesBuilder import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat @@ -54,8 +53,7 @@ case class GlutenCacheFilesCommand( override def run(session: SparkSession): Seq[Row] = { if ( !session.sparkContext.getConf.getBoolean( - s"${GlutenConfig.GLUTEN_CONFIG_PREFIX}${CHBackend.BACKEND_NAME}" + - s".runtime_config.gluten_cache.local.enabled", + CHConf.runtimeConfig("gluten_cache.local.enabled"), defaultValue = false) ) { return Seq(Row(false, "Config `gluten_cache.local.enabled` is disabled.")) diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala index ec7c686707aa..e300533bdb22 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.utils -import org.apache.gluten.backendsapi.clickhouse.CHBackend +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution.{GlutenMergeTreePartition, MergeTreePartRange, MergeTreePartSplit} import org.apache.gluten.expression.{ConverterUtils, ExpressionConverter} import org.apache.gluten.softaffinity.SoftAffinityManager @@ -171,9 +171,10 @@ object MergeTreePartsPartitionsUtil extends Logging { val ret = ClickhouseSnapshot.pathToAddMTPCache.getIfPresent(path) if (ret == null) { val keys = ClickhouseSnapshot.pathToAddMTPCache.asMap().keySet() - val keySample = keys.isEmpty match { - case true => "" - case false => keys.iterator().next() + val keySample = if (keys.isEmpty) { + "" + } else { + keys.iterator().next() } throw new IllegalStateException( "Can't find AddMergeTreeParts from cache pathToAddMTPCache for key: " + @@ -418,7 +419,7 @@ object MergeTreePartsPartitionsUtil extends Logging { bucketId => val currBucketParts: Seq[MergeTreePartRange] = prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty) - if (!currBucketParts.isEmpty) { + if (currBucketParts.nonEmpty) { val currentFiles = currBucketParts.map { part => MergeTreePartSplit( @@ -453,8 +454,7 @@ object MergeTreePartsPartitionsUtil extends Logging { } private def useDriverFilter(filterExprs: Seq[Expression], sparkSession: SparkSession): Boolean = { - val enableDriverFilterKey = s"${CHBackend.CONF_PREFIX}.runtime_settings" + - s".enabled_driver_filter_mergetree_index" + val enableDriverFilterKey = CHConf.runtimeSettings("enabled_driver_filter_mergetree_index") // When using soft affinity, disable driver filter filterExprs.nonEmpty && sparkSession.sessionState.conf.getConfString( diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala index 232e9ec10c5b..53ccb8d1c046 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.datasources.v2.clickhouse +import org.apache.gluten.backendsapi.clickhouse.CHConf + import org.apache.spark.sql.catalyst.catalog.BucketSpec import java.util @@ -28,18 +30,16 @@ object ClickHouseConfig { val NAME = "clickhouse" val ALT_NAME = "clickhouse" val METADATA_DIR = "_delta_log" - val FORMAT_ENGINE = "engine" - val DEFAULT_ENGINE = "mergetree" - val OPT_NAME_PREFIX = "clickhouse." + private val FORMAT_ENGINE = "engine" + private val DEFAULT_ENGINE = "mergetree" + private val OPT_NAME_PREFIX = "clickhouse." @deprecated // Whether to use MergeTree DataSource V2 API, default is false, fall back to V1. - val USE_DATASOURCE_V2 = "spark.gluten.sql.columnar.backend.ch.use.v2" + val USE_DATASOURCE_V2: String = CHConf.prefixOf("use.v2") val DEFAULT_USE_DATASOURCE_V2 = "false" - val CLICKHOUSE_WORKER_ID = "spark.gluten.sql.columnar.backend.ch.worker.id" - - val CLICKHOUSE_WAREHOUSE_DIR = "spark.gluten.sql.columnar.backend.ch.warehouse.dir" + val CLICKHOUSE_WORKER_ID: String = CHConf.prefixOf("worker.id") /** Create a mergetree configurations and returns the normalized key -> value map. */ def createMergeTreeConfigurations( @@ -53,8 +53,11 @@ object ClickHouseConfig { if (!configurations.contains(FORMAT_ENGINE)) { configurations += (FORMAT_ENGINE -> DEFAULT_ENGINE) } else { - val engineValue = configurations.get(FORMAT_ENGINE) - if (!engineValue.equals(DEFAULT_ENGINE) && !engineValue.equals("parquet")) { + if ( + !configurations + .get(FORMAT_ENGINE) + .exists(s => s.equals(DEFAULT_ENGINE) || s.equals("parquet")) + ) { configurations += (FORMAT_ENGINE -> DEFAULT_ENGINE) } } @@ -80,8 +83,7 @@ object ClickHouseConfig { } def isMergeTreeFormatEngine(configuration: Map[String, String]): Boolean = { - configuration.contains(FORMAT_ENGINE) && - configuration.get(FORMAT_ENGINE).get.equals(DEFAULT_ENGINE) + configuration.get(FORMAT_ENGINE).exists(_.equals(DEFAULT_ENGINE)) } /** Get the related clickhouse option when using DataFrameWriter / DataFrameReader */ diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/RunTPCHTest.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/RunTPCHTest.scala index 6dfab5bf5a91..4c774c9e037b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/RunTPCHTest.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/RunTPCHTest.scala @@ -16,9 +16,11 @@ */ package org.apache.gluten +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.benchmarks.GenTPCHTableScripts import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.commons.io.FileUtils @@ -34,10 +36,10 @@ object RunTPCHTest { // parquet or mergetree val fileFormat = "parquet" val libPath = "/usr/local/clickhouse/lib/libch.so" - if (!(new File(libPath)).exists()) System.exit(1) + if (!new File(libPath).exists()) System.exit(1) // TPCH data files path val dataFilesPath = "/data/tpch-data/" + fileFormat - if (!(new File(dataFilesPath)).exists()) System.exit(1) + if (!new File(dataFilesPath).exists()) System.exit(1) // the time of execution val executedCnt = 5 // local thread count @@ -91,7 +93,7 @@ object RunTPCHTest { .config("spark.databricks.delta.properties.defaults.checkpointInterval", 5) .config("spark.databricks.delta.stalenessLimit", 3600 * 1000) .config("spark.gluten.sql.columnar.columnarToRow", columnarColumnToRow) - .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .config(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .config(GlutenConfig.GLUTEN_LIB_PATH, libPath) .config("spark.gluten.sql.columnar.iterator", "true") .config("spark.gluten.sql.columnar.hashagg.enablefinal", "true") @@ -99,7 +101,7 @@ object RunTPCHTest { .config("spark.sql.columnVector.offheap.enabled", "true") .config("spark.memory.offHeap.enabled", "true") .config("spark.memory.offHeap.size", offHeapSize) - .config("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") + .config(CHConf.runtimeConfig("logger.level"), "error") .config("spark.sql.warehouse.dir", warehouse) .config( "javax.jdo.option.ConnectionURL", diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarMemorySortShuffleSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarMemorySortShuffleSuite.scala index 1b79e296595f..055b5c807d37 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarMemorySortShuffleSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarMemorySortShuffleSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.gluten.execution +import org.apache.gluten.backendsapi.clickhouse.CHConf + import org.apache.spark.SparkConf import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -35,7 +37,9 @@ class GlutenClickHouseColumnarMemorySortShuffleSuite .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.forceMemorySortShuffle", "true") + .set(CHConf.prefixOf("forceMemorySortShuffle"), "true") + + // TODO: forceMemorySortShuffle } test("TPCH Q1") { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarShuffleAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarShuffleAQESuite.scala index 0ac6284991ae..b43fc2625f0b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarShuffleAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarShuffleAQESuite.scala @@ -16,6 +16,8 @@ */ package org.apache.gluten.execution +import org.apache.gluten.backendsapi.clickhouse.CHConf + import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.optimizer._ @@ -32,7 +34,6 @@ class GlutenClickHouseColumnarShuffleAQESuite override protected val tablesPath: String = basePath + "/tpch-data-ch" override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" override protected val queriesResults: String = rootPath + "mergetree-queries-output" - private val backendConfigPrefix = "spark.gluten.sql.columnar.backend.ch." /** Run Gluten + ClickHouse Backend with ColumnarShuffleManager */ override protected def sparkConf: SparkConf = { @@ -53,13 +54,11 @@ class GlutenClickHouseColumnarShuffleAQESuite case csr: AQEShuffleReadExec => csr } assert(colCustomShuffleReaderExecs.size == 2) - val coalescedPartitionSpec0 = colCustomShuffleReaderExecs(0) - .partitionSpecs(0) + val coalescedPartitionSpec0 = colCustomShuffleReaderExecs.head.partitionSpecs.head .asInstanceOf[CoalescedPartitionSpec] assert(coalescedPartitionSpec0.startReducerIndex == 0) assert(coalescedPartitionSpec0.endReducerIndex == 5) - val coalescedPartitionSpec1 = colCustomShuffleReaderExecs(1) - .partitionSpecs(0) + val coalescedPartitionSpec1 = colCustomShuffleReaderExecs(1).partitionSpecs.head .asInstanceOf[CoalescedPartitionSpec] assert(coalescedPartitionSpec1.startReducerIndex == 0) assert(coalescedPartitionSpec1.endReducerIndex == 5) @@ -180,7 +179,7 @@ class GlutenClickHouseColumnarShuffleAQESuite test("GLUTEN-6768 rerorder hash join") { withSQLConf( - ("spark.gluten.sql.columnar.backend.ch.enable_reorder_hash_join_tables", "true"), + (CHConf.prefixOf("enable_reorder_hash_join_tables"), "true"), ("spark.sql.adaptive.enabled", "true")) { spark.sql("create table t1(a int, b int) using parquet") spark.sql("create table t2(a int, b int) using parquet") @@ -266,8 +265,8 @@ class GlutenClickHouseColumnarShuffleAQESuite test("GLUTEN-6768 change mixed join condition into multi join on clauses") { withSQLConf( - (backendConfigPrefix + "runtime_config.prefer_multi_join_on_clauses", "true"), - (backendConfigPrefix + "runtime_config.multi_join_on_clauses_build_side_row_limit", "1000000") + (CHConf.runtimeConfig("prefer_multi_join_on_clauses"), "true"), + (CHConf.runtimeConfig("multi_join_on_clauses_build_side_row_limit"), "1000000") ) { spark.sql("create table t1(a int, b int, c int, d int) using parquet") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala index 3528bc12b264..9fe09634d776 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala @@ -18,7 +18,6 @@ package org.apache.gluten.execution import org.apache.spark.SparkConf import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.delta.files.TahoeFileIndex import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -39,6 +38,8 @@ class GlutenClickHouseDeltaParquetWriteSuite override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" override protected val queriesResults: String = rootPath + "mergetree-queries-output" + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { super.sparkConf @@ -50,13 +51,8 @@ class GlutenClickHouseDeltaParquetWriteSuite .set("spark.sql.files.maxPartitionBytes", "20000000") .set("spark.gluten.sql.native.writer.enabled", "true") .set("spark.sql.storeAssignmentPolicy", "legacy") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", - "false") - .set( - "spark.databricks.delta.retentionDurationCheck.enabled", - "false" - ) + .setCHSettings("mergetree.merge_after_insert", false) + .set("spark.databricks.delta.retentionDurationCheck.enabled", "false") } override protected def createTPCHNotNullTables(): Unit = { @@ -128,14 +124,14 @@ class GlutenClickHouseDeltaParquetWriteSuite case f: FileSourceScanExecTransformer => f case w: WholeStageTransformer => w } - assert(plans.size == 4) + assert(plans.size === 4) val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] assert(parquetScan.nodeName.startsWith("Scan parquet ")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) - assert(addFiles.size == 5) + val addFiles = fileIndex.matchingFiles(Nil, Nil) + assert(addFiles.size === 5) } } @@ -183,7 +179,7 @@ class GlutenClickHouseDeltaParquetWriteSuite |""".stripMargin assert( // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) == 300001 + spark.sql(sql2).collect().apply(0).get(0) === 300001 ) } @@ -235,7 +231,7 @@ class GlutenClickHouseDeltaParquetWriteSuite |""".stripMargin assert( // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) == 2418 + spark.sql(sql2).collect().apply(0).get(0) === 2418 ) } @@ -288,7 +284,7 @@ class GlutenClickHouseDeltaParquetWriteSuite |""".stripMargin assert( // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) == 21875 + spark.sql(sql2).collect().apply(0).get(0) === 21875 ) } } @@ -343,19 +339,19 @@ class GlutenClickHouseDeltaParquetWriteSuite val result = df.collect() assert( // in test data, there are only 1 row with l_orderkey = 12647 - result.apply(0).get(0) == 1 + result.apply(0).get(0) === 1 ) val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f } - assert(scanExec.size == 1) + assert(scanExec.size === 1) val parquetScan = scanExec.head assert(parquetScan.nodeName.startsWith("Scan parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) - assert(addFiles.size == 6) + val addFiles = fileIndex.matchingFiles(Nil, Nil) + assert(addFiles.size === 6) } val sql2 = @@ -365,7 +361,7 @@ class GlutenClickHouseDeltaParquetWriteSuite |""".stripMargin assert( // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) == 600572 + spark.sql(sql2).collect().apply(0).get(0) === 600572 ) } @@ -412,15 +408,15 @@ class GlutenClickHouseDeltaParquetWriteSuite |""".stripMargin) val result = df.collect() assert( - result.apply(0).get(0) == 1802445 + result.apply(0).get(0) === 1802445 ) val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f } val parquetScan = scanExec.head val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) - assert(addFiles.size == 6) + val addFiles = fileIndex.matchingFiles(Nil, Nil) + assert(addFiles.size === 6) } { @@ -431,7 +427,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | select sum(l_linenumber) from lineitem_delta_parquet_delete |""".stripMargin) assert( - df3.collect().apply(0).get(0) == 1200671 + df3.collect().apply(0).get(0) === 1200671 ) } } @@ -475,7 +471,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | select sum(l_linenumber) from lineitem_delta_parquet_upsert |""".stripMargin) assert( - df0.collect().apply(0).get(0) == 1802446 + df0.collect().apply(0).get(0) === 1802446 ) } @@ -514,7 +510,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | select count(*) from $tableName |""".stripMargin) assert( - df1.collect().apply(0).get(0) == 600572 + 3506 + df1.collect().apply(0).get(0) === 600572 + 3506 ) } { @@ -523,7 +519,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | select count(*) from $tableName where l_returnflag = 'Z' |""".stripMargin) assert( - df2.collect().apply(0).get(0) == 3506 + df2.collect().apply(0).get(0) === 3506 ) } @@ -533,7 +529,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | select count(*) from $tableName where l_orderkey > 10000000 |""".stripMargin) assert( - df3.collect().apply(0).get(0) == 3506 + df3.collect().apply(0).get(0) === 3506 ) } } @@ -666,34 +662,31 @@ class GlutenClickHouseDeltaParquetWriteSuite runTPCHQueryBySQL(1, sqlStr, compareResult = false) { df => val result = df.collect() - assert(result.size == 2) + assert(result.length === 2) assert(result(0).getString(0).equals("A")) assert(result(0).getString(1).equals("F")) - assert(result(0).getDouble(2) == 368009.0) + assert(result(0).getDouble(2) === 368009.0) assert(result(1).getString(0).equals("R")) assert(result(1).getString(1).equals("F")) - assert(result(1).getDouble(2) == 312371.0) + assert(result(1).getDouble(2) === 312371.0) val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f } - assert(scanExec.size == 1) + assert(scanExec.size === 1) - val parquetScan = scanExec(0) + val parquetScan = scanExec.head assert(parquetScan.nodeName.startsWith("Scan parquet")) - assert(parquetScan.metrics("numFiles").value == 201) + assert(parquetScan.metrics("numFiles").value === 201) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) - - assert(addFiles.size == 201) - assert( - addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1993-03-31")).size == 2) - assert( - addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1993-01-01")).size == 4) - assert( - addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1993-02-21")).size == 3) + val addFiles = fileIndex.matchingFiles(Nil, Nil) + + assert(addFiles.size === 201) + assert(addFiles.count(_.partitionValues("l_shipdate").equals("1993-03-31")) === 2) + assert(addFiles.count(_.partitionValues("l_shipdate").equals("1993-01-01")) === 4) + assert(addFiles.count(_.partitionValues("l_shipdate").equals("1993-02-21")) === 3) } } @@ -739,14 +732,14 @@ class GlutenClickHouseDeltaParquetWriteSuite val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f } - assert(scanExec.size == 1) + assert(scanExec.size === 1) - val parquetScan = scanExec(0) + val parquetScan = scanExec.head assert(parquetScan.nodeName.startsWith("Scan parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) - assert(addFiles.size == 1) + val addFiles = fileIndex.matchingFiles(Nil, Nil) + assert(addFiles.size === 1) } } @@ -866,14 +859,14 @@ class GlutenClickHouseDeltaParquetWriteSuite case f: FileSourceScanExecTransformer => f case w: WholeStageTransformer => w } - assert(plans.size == 4) + assert(plans.size === 4) val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] assert(parquetScan.nodeName.startsWith("Scan parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) - assert(addFiles.size == 1) + val addFiles = fileIndex.matchingFiles(Nil, Nil) + assert(addFiles.size === 1) } val result = spark.read @@ -881,7 +874,7 @@ class GlutenClickHouseDeltaParquetWriteSuite .load(dataPath) .where("l_shipdate = date'1998-09-02'") .count() - assert(result == 183) + assert(result === 183) } test( @@ -914,7 +907,7 @@ class GlutenClickHouseDeltaParquetWriteSuite .format("delta") .load(dataPath) .count() - assert(result == 2418) + assert(result === 2418) } test( @@ -948,7 +941,7 @@ class GlutenClickHouseDeltaParquetWriteSuite .format("delta") .load(dataPath) .count() - assert(result == 21875) + assert(result === 21875) } } @@ -974,18 +967,18 @@ class GlutenClickHouseDeltaParquetWriteSuite .format("delta") .load(dataPath) .where("l_returnflag = 'Z'") - assert(df.count() == 1) + assert(df.count() === 1) val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f } - assert(scanExec.size == 1) + assert(scanExec.size === 1) val parquetScan = scanExec.head assert(parquetScan.nodeName.startsWith("Scan parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) - assert(addFiles.size == 6) + val addFiles = fileIndex.matchingFiles(Nil, Nil) + assert(addFiles.size === 6) } val clickhouseTable = DeltaTable.forPath(spark, dataPath) @@ -996,24 +989,24 @@ class GlutenClickHouseDeltaParquetWriteSuite .format("delta") .load(dataPath) .where("l_returnflag = 'X'") - assert(df.count() == 1) + assert(df.count() === 1) val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f } - assert(scanExec.size == 1) + assert(scanExec.size === 1) val parquetScan = scanExec.head assert(parquetScan.nodeName.startsWith("Scan parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) - assert(addFiles.size == 6) + val addFiles = fileIndex.matchingFiles(Nil, Nil) + assert(addFiles.size === 6) } val df = spark.read .format("delta") .load(dataPath) - assert(df.count() == 600572) + assert(df.count() === 600572) } test("test path based parquet delete with the delta") { @@ -1035,21 +1028,21 @@ class GlutenClickHouseDeltaParquetWriteSuite val df = spark.read .format("delta") .load(dataPath) - assert(df.count() == 600571) + assert(df.count() === 600571) val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f } val parquetScan = scanExec.head val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) - assert(addFiles.size == 6) + val addFiles = fileIndex.matchingFiles(Nil, Nil) + assert(addFiles.size === 6) val clickhouseTable = DeltaTable.forPath(spark, dataPath) clickhouseTable.delete("mod(l_orderkey, 3) = 2") val df1 = spark.read .format("delta") .load(dataPath) - assert(df1.count() == 400089) + assert(df1.count() === 400089) } test("test path based parquet upsert with the delta") { @@ -1069,7 +1062,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | select count(*) from delta.`$dataPath` |""".stripMargin) assert( - df0.collect().apply(0).get(0) == 600572 + df0.collect().apply(0).get(0) === 600572 ) upsertPathBasedSourceTableAndCheck(dataPath) } @@ -1106,7 +1099,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | select count(*) from delta.`$dataPath` |""".stripMargin) assert( - df1.collect().apply(0).get(0) == 600572 + 3506 + df1.collect().apply(0).get(0) === 600572 + 3506 ) } { @@ -1115,7 +1108,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | select count(*) from delta.`$dataPath` where l_returnflag = 'Z' |""".stripMargin) assert( - df2.collect().apply(0).get(0) == 3506 + df2.collect().apply(0).get(0) === 3506 ) } @@ -1125,7 +1118,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | select count(*) from delta.`$dataPath` where l_orderkey > 10000000 |""".stripMargin) assert( - df3.collect().apply(0).get(0) == 3506 + df3.collect().apply(0).get(0) === 3506 ) } } @@ -1183,32 +1176,30 @@ class GlutenClickHouseDeltaParquetWriteSuite runTPCHQueryBySQL(1, sqlStr, compareResult = false) { df => val result = df.collect() - assert(result.size == 2) + assert(result.length === 2) assert(result(0).getString(0).equals("A")) assert(result(0).getString(1).equals("F")) - assert(result(0).getDouble(2) == 306633.0) + assert(result(0).getDouble(2) === 306633.0) assert(result(1).getString(0).equals("R")) assert(result(1).getString(1).equals("F")) - assert(result(1).getDouble(2) == 312371.0) + assert(result(1).getDouble(2) === 312371.0) val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f } - assert(scanExec.size == 1) + assert(scanExec.size === 1) - val parquetScan = scanExec(0) + val parquetScan = scanExec.head assert(parquetScan.nodeName.startsWith("Scan parquet")) - assert(parquetScan.metrics("numFiles").value == 200) + assert(parquetScan.metrics("numFiles").value === 200) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + val addFiles = fileIndex.matchingFiles(Nil, Nil) - assert(addFiles.size == 200) - assert( - addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1993-03-31")).size == 2) - assert( - addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1993-01-01")).size == 4) + assert(addFiles.size === 200) + assert(addFiles.count(_.partitionValues("l_shipdate").equals("1993-03-31")) === 2) + assert(addFiles.count(_.partitionValues("l_shipdate").equals("1993-01-01")) === 4) } } @@ -1266,10 +1257,10 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql("optimize lineitem_delta_parquet_optimize") val ret = spark.sql("select count(*) from lineitem_delta_parquet_optimize").collect() - assert(ret.apply(0).get(0) == 600572) + assert(ret.apply(0).get(0) === 600572) assert( - countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize")) == 24 + countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize")) === 24 ) } } @@ -1302,27 +1293,27 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sparkContext.setJobGroup("test3", "test3") spark.sql("optimize lineitem_delta_parquet_optimize_p2") val job_ids = spark.sparkContext.statusTracker.getJobIdsForGroup("test3") - if (sparkVersion.equals("3.2")) { - assert(job_ids.size == 7) // WILL trigger actual merge job + if (spark32) { + assert(job_ids.length === 7) // WILL trigger actual merge job } else { - assert(job_ids.size == 8) // WILL trigger actual merge job + assert(job_ids.length === 8) // WILL trigger actual merge job } spark.sparkContext.clearJobGroup() val ret = spark.sql("select count(*) from lineitem_delta_parquet_optimize_p2").collect() - assert(ret.apply(0).get(0) == 600572) + assert(ret.apply(0).get(0) === 600572) - assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p2")) == 23) + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p2")) === 23) spark.sql("VACUUM lineitem_delta_parquet_optimize_p2 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { - assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p2")) == 5) + if (spark32) { + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p2")) === 5) } else { - assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p2")) == 7) + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p2")) === 7) } val ret2 = spark.sql("select count(*) from lineitem_delta_parquet_optimize_p2").collect() - assert(ret2.apply(0).get(0) == 600572) + assert(ret2.apply(0).get(0) === 600572) } testSparkVersionLE33("test parquet optimize parallel delete") { @@ -1341,18 +1332,18 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql("optimize lineitem_delta_parquet_optimize_p4") val ret = spark.sql("select count(*) from lineitem_delta_parquet_optimize_p4").collect() - assert(ret.apply(0).get(0) == 600572) + assert(ret.apply(0).get(0) === 600572) - assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p4")) == 149) + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p4")) === 149) spark.sql("VACUUM lineitem_delta_parquet_optimize_p4 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { - assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p4")) == 23) + if (spark32) { + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p4")) === 23) } else { - assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p4")) == 25) + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p4")) === 25) } val ret2 = spark.sql("select count(*) from lineitem_delta_parquet_optimize_p4").collect() - assert(ret2.apply(0).get(0) == 600572) + assert(ret2.apply(0).get(0) === 600572) } } @@ -1360,8 +1351,8 @@ class GlutenClickHouseDeltaParquetWriteSuite val dataPath = s"$basePath/lineitem_delta_parquet_optimize_path_based" clearDataPath(dataPath) withSQLConf( - ("spark.databricks.delta.optimize.maxFileSize" -> "1000000"), - ("spark.databricks.delta.optimize.minFileSize" -> "838000")) { + "spark.databricks.delta.optimize.maxFileSize" -> "1000000", + "spark.databricks.delta.optimize.minFileSize" -> "838000") { val sourceDF = spark.sql(s""" |select /*+ REPARTITION(50) */ * from lineitem @@ -1378,32 +1369,32 @@ class GlutenClickHouseDeltaParquetWriteSuite clickhouseTable.optimize().executeCompaction() clickhouseTable.vacuum(0.0) - if (sparkVersion.equals("3.2")) { + if (spark32) { assert(countFiles(new File(dataPath)) === 27) } else { assert(countFiles(new File(dataPath)) === 29) } val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() - assert(ret.apply(0).get(0) == 600572) + assert(ret.apply(0).get(0) === 600572) } withSQLConf( - ("spark.databricks.delta.optimize.maxFileSize" -> "10000000"), - ("spark.databricks.delta.optimize.minFileSize" -> "1000000")) { + "spark.databricks.delta.optimize.maxFileSize" -> "10000000", + "spark.databricks.delta.optimize.minFileSize" -> "1000000") { val clickhouseTable = DeltaTable.forPath(spark, dataPath) clickhouseTable.optimize().executeCompaction() clickhouseTable.vacuum(0.0) - if (sparkVersion.equals("3.2")) { - assert(countFiles(new File(dataPath)) == 6) + if (spark32) { + assert(countFiles(new File(dataPath)) === 6) } else { - assert(countFiles(new File(dataPath)) == 12) + assert(countFiles(new File(dataPath)) === 12) } val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() - assert(ret.apply(0).get(0) == 600572) + assert(ret.apply(0).get(0) === 600572) } // now merge all parts (testing merging from merged parts) @@ -1411,14 +1402,14 @@ class GlutenClickHouseDeltaParquetWriteSuite clickhouseTable.optimize().executeCompaction() clickhouseTable.vacuum(0.0) - if (sparkVersion.equals("3.2")) { - assert(countFiles(new File(dataPath)) == 5) + if (spark32) { + assert(countFiles(new File(dataPath)) === 5) } else { - assert(countFiles(new File(dataPath)) == 13) + assert(countFiles(new File(dataPath)) === 13) } val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() - assert(ret.apply(0).get(0) == 600572) + assert(ret.apply(0).get(0) === 600572) } } // scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala index 5b499026f81c..bb99c6bd1ef0 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.gluten.execution +import org.apache.gluten.backendsapi.clickhouse.CHConf + import org.apache.spark.SparkConf import org.apache.spark.sql.{functions, DataFrame, Row} import org.apache.spark.sql.execution.LocalTableScanExec @@ -64,12 +66,12 @@ class GlutenClickHouseFileFormatSuite override protected def createTPCHNotNullTables(): Unit = {} override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.sql.adaptive.enabled", "true") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.date_time_input_format", - "best_effort_us") - .set("spark.gluten.sql.columnar.backend.ch.runtime_settings.use_excel_serialization", "true") + .setCHSettings("date_time_input_format", "best_effort_us") + .setCHSettings("use_excel_serialization", true) } // in this case, FakeRowAdaptor does R2C @@ -877,10 +879,7 @@ class GlutenClickHouseFileFormatSuite .toDF() .createTempView("no_quote_table") - withSQLConf(( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.use_excel_serialization.quote_strict", - "true" - )) { + withSQLConf((CHConf.runtimeSettings("use_excel_serialization.quote_strict"), "true")) { compareResultsAgainstVanillaSpark( "select * from no_quote_table", compareResult = true, @@ -1186,11 +1185,7 @@ class GlutenClickHouseFileFormatSuite } test("issue-2881 null string test") { - withSQLConf( - ( - "spark.gluten.sql.columnar.backend.ch.runtime_settings." + - "use_excel_serialization.empty_as_null", - "true")) { + withSQLConf((CHConf.runtimeSettings("use_excel_serialization.empty_as_null"), "true")) { val file_path = csvDataPath + "/null_string.csv" val schema = StructType.apply( Seq( @@ -1223,11 +1218,7 @@ class GlutenClickHouseFileFormatSuite } test("issue-3542 null string test") { - withSQLConf( - ( - "spark.gluten.sql.columnar.backend.ch.runtime_settings." + - "use_excel_serialization.empty_as_null", - "false")) { + withSQLConf((CHConf.runtimeSettings("use_excel_serialization.empty_as_null"), "false")) { val file_path = csvDataPath + "/null_string.csv" val schema = StructType.apply( Seq( @@ -1312,11 +1303,7 @@ class GlutenClickHouseFileFormatSuite } test("issues-3609 int read test") { - withSQLConf( - ( - "spark.gluten.sql.columnar.backend.ch.runtime_settings." + - "use_excel_serialization.number_force", - "false")) { + withSQLConf((CHConf.runtimeSettings("use_excel_serialization.number_force"), "false")) { val csv_path = csvDataPath + "/int_special.csv" val options = new util.HashMap[String, String]() options.put("delimiter", ",") @@ -1345,11 +1332,7 @@ class GlutenClickHouseFileFormatSuite checkAnswer(df, expectedAnswer) } - withSQLConf( - ( - "spark.gluten.sql.columnar.backend.ch.runtime_settings." + - "use_excel_serialization.number_force", - "true")) { + withSQLConf((CHConf.runtimeSettings("use_excel_serialization.number_force"), "true")) { val csv_path = csvDataPath + "/int_special.csv" val options = new util.HashMap[String, String]() options.put("delimiter", ",") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseJoinSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseJoinSuite.scala index 75c4372a04d9..97ba1fe4214c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseJoinSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseJoinSuite.scala @@ -17,9 +17,11 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig class GlutenClickHouseJoinSuite extends GlutenClickHouseWholeStageTransformerSuite { @@ -28,7 +30,7 @@ class GlutenClickHouseJoinSuite extends GlutenClickHouseWholeStageTransformerSui rootPath + "../../../../gluten-core/src/test/resources/tpch-queries" protected val queriesResults: String = rootPath + "queries-output" - private val joinAlgorithm = "spark.gluten.sql.columnar.backend.ch.runtime_settings.join_algorithm" + private val joinAlgorithm = CHConf.runtimeSettings("join_algorithm") override protected def sparkConf: SparkConf = { super.sparkConf @@ -38,7 +40,7 @@ class GlutenClickHouseJoinSuite extends GlutenClickHouseWholeStageTransformerSui .set("spark.sql.adaptive.enabled", "false") .set("spark.sql.files.minPartitionNum", "1") .set("spark.gluten.sql.columnar.columnartorow", "true") - .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseS3SourceSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseS3SourceSuite.scala index ac33757032e1..aa5123440d3f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseS3SourceSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseS3SourceSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.gluten.execution +import org.apache.gluten.backendsapi.clickhouse.CHConf + import org.apache.spark.SparkConf // Some sqls' line length exceeds 100 @@ -31,6 +33,8 @@ class GlutenClickHouseS3SourceSuite extends GlutenClickHouseTPCHAbstractSuite { override protected val queriesResults: String = rootPath + "queries-output" override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") @@ -44,10 +48,8 @@ class GlutenClickHouseS3SourceSuite extends GlutenClickHouseTPCHAbstractSuite { .set("spark.hadoop.fs.s3a.path.style.access", "true") .set("spark.hadoop.fs.s3a.connection.ssl.enabled", "false") .set("spark.hadoop.fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.s3.local_cache.enabled", "true") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.s3.local_cache.cache_path", - "/data/gluten-ch-cache-dir") + .setCHConfig("s3.local_cache.enabled", true) + .setCHConfig("s3.local_cache.cache_path", "/data/gluten-ch-cache-dir") } override protected val createNullableTables = true @@ -93,14 +95,13 @@ class GlutenClickHouseS3SourceSuite extends GlutenClickHouseTPCHAbstractSuite { println(s"currTime=$currTime") // scalastyle:on println spark.sparkContext.setLocalProperty( - "spark.gluten.sql.columnar.backend.ch." + - "runtime_settings.spark.kylin.local-cache.accept-cache-time", + CHConf.runtimeSettings("spark.kylin.local-cache.accept-cache-time"), currTime.toString) spark .sql(""" |select * from supplier_s3 |""".stripMargin) - .show(10, false) + .show(10, truncate = false) Thread.sleep(5000) @@ -108,14 +109,13 @@ class GlutenClickHouseS3SourceSuite extends GlutenClickHouseTPCHAbstractSuite { println(s"currTime=$currTime") // scalastyle:on println spark.sparkContext.setLocalProperty( - "spark.gluten.sql.columnar.backend.ch." + - "runtime_settings.spark.kylin.local-cache.accept-cache-time", + CHConf.runtimeSettings("spark.kylin.local-cache.accept-cache-time"), currTime.toString) spark .sql(""" |select * from supplier_s3 |""".stripMargin) - .show(10, false) + .show(10, truncate = false) Thread.sleep(5000) currTime = System.currentTimeMillis() @@ -123,14 +123,13 @@ class GlutenClickHouseS3SourceSuite extends GlutenClickHouseTPCHAbstractSuite { println(s"currTime=$currTime") // scalastyle:on println spark.sparkContext.setLocalProperty( - "spark.gluten.sql.columnar.backend.ch." + - "runtime_settings.spark.kylin.local-cache.accept-cache-time", + CHConf.runtimeSettings("spark.kylin.local-cache.accept-cache-time"), currTime.toString) spark .sql(""" |select * from supplier_s3 |""".stripMargin) - .show(10, false) + .show(10, truncate = false) } } // scalastyle:on line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseSyntheticDataSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseSyntheticDataSuite.scala index 8db4e3b10fe2..bd739a291e51 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseSyntheticDataSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseSyntheticDataSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import java.time.LocalDate @@ -51,7 +52,7 @@ class GlutenClickHouseSyntheticDataSuite .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set("spark.gluten.sql.columnar.columnarToRow", "true") - .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") @@ -136,7 +137,7 @@ class GlutenClickHouseSyntheticDataSuite prepareTables() var sqlStr: String = null - var expected: Seq[Row] = null; + var expected: Seq[Row] = null withSQLConf(vanillaSparkConfs(): _*) { val supportedAggs = "count" :: "avg" :: "sum" :: "min" :: "max" :: Nil val selected = supportedAggs @@ -173,7 +174,7 @@ class GlutenClickHouseSyntheticDataSuite test("test data function in https://github.com/Kyligence/ClickHouse/issues/88") { var sqlStr: String = null - var expected: Seq[Row] = null; + var expected: Seq[Row] = null val x = spark import x.implicits._ @@ -205,7 +206,7 @@ class GlutenClickHouseSyntheticDataSuite test("sql on Seq based(row based) DataFrame") { var sqlStr: String = null - var expected: Seq[Row] = null; + var expected: Seq[Row] = null val x = spark import x.implicits._ diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala index abb7d27ffe92..f1d790322fe6 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.delta.{ClickhouseSnapshot, DeltaLog} +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.types.{StructField, StructType} import org.apache.commons.io.FileUtils @@ -131,7 +132,7 @@ abstract class GlutenClickHouseTPCDSAbstractSuite .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set("spark.gluten.sql.columnar.columnarToRow", "true") - .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala index 8d671e29f18b..7f15edf73b85 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.sql.DataFrame import org.apache.spark.sql.delta.{ClickhouseSnapshot, DeltaLog} +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.commons.io.FileUtils import org.scalatest.time.SpanSugar.convertIntToGrainOfTime @@ -567,7 +568,7 @@ abstract class GlutenClickHouseTPCHAbstractSuite .set("spark.databricks.delta.stalenessLimit", "3600000") .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") .set("spark.gluten.sql.columnar.columnarToRow", "true") - .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") @@ -588,7 +589,7 @@ abstract class GlutenClickHouseTPCHAbstractSuite assert(CHBroadcastBuildSideCache.size() <= 10) } - ClickhouseSnapshot.clearAllFileStatusCache + ClickhouseSnapshot.clearAllFileStatusCache() DeltaLog.clearCache() super.afterAll() // init GlutenConfig in the next beforeAll diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala index e05cf7274fef..6a09bf494217 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala @@ -87,84 +87,80 @@ class GlutenClickHouseTPCHBucketSuite | USING clickhouse | LOCATION '$customerData' | CLUSTERED BY (c_custkey) - | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (c_custkey)"} INTO 2 BUCKETS; + | ${if (spark32) "" else "SORTED BY (c_custkey)"} INTO 2 BUCKETS; |""".stripMargin) val lineitemData = tablesPath + "/lineitem" spark.sql(s"DROP TABLE IF EXISTS lineitem") - spark.sql( - s""" - | CREATE EXTERNAL TABLE IF NOT EXISTS lineitem ( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string) - | USING clickhouse - | LOCATION '$lineitemData' - | CLUSTERED BY (l_orderkey) - | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_shipdate, l_orderkey)"} INTO 2 BUCKETS; - |""".stripMargin) + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS lineitem ( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string) + | USING clickhouse + | LOCATION '$lineitemData' + | CLUSTERED BY (l_orderkey) + | ${if (spark32) "" else "SORTED BY (l_shipdate, l_orderkey)"} INTO 2 BUCKETS; + |""".stripMargin) val nationData = tablesPath + "/nation" spark.sql(s"DROP TABLE IF EXISTS nation") - spark.sql( - s""" - | CREATE EXTERNAL TABLE IF NOT EXISTS nation ( - | n_nationkey bigint, - | n_name string, - | n_regionkey bigint, - | n_comment string) - | USING clickhouse - | LOCATION '$nationData' - | CLUSTERED BY (n_nationkey) - | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (n_nationkey)"} INTO 1 BUCKETS; - |""".stripMargin) + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS nation ( + | n_nationkey bigint, + | n_name string, + | n_regionkey bigint, + | n_comment string) + | USING clickhouse + | LOCATION '$nationData' + | CLUSTERED BY (n_nationkey) + | ${if (spark32) "" else "SORTED BY (n_nationkey)"} INTO 1 BUCKETS; + |""".stripMargin) val regionData = tablesPath + "/region" spark.sql(s"DROP TABLE IF EXISTS region") - spark.sql( - s""" - | CREATE EXTERNAL TABLE IF NOT EXISTS region ( - | r_regionkey bigint, - | r_name string, - | r_comment string) - | USING clickhouse - | LOCATION '$regionData' - | CLUSTERED BY (r_regionkey) - | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (r_regionkey)"} INTO 1 BUCKETS; - |""".stripMargin) + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS region ( + | r_regionkey bigint, + | r_name string, + | r_comment string) + | USING clickhouse + | LOCATION '$regionData' + | CLUSTERED BY (r_regionkey) + | ${if (spark32) "" else "SORTED BY (r_regionkey)"} INTO 1 BUCKETS; + |""".stripMargin) val ordersData = tablesPath + "/orders" spark.sql(s"DROP TABLE IF EXISTS orders") - spark.sql( - s""" - | CREATE EXTERNAL TABLE IF NOT EXISTS orders ( - | o_orderkey bigint, - | o_custkey bigint, - | o_orderstatus string, - | o_totalprice double, - | o_orderdate date, - | o_orderpriority string, - | o_clerk string, - | o_shippriority bigint, - | o_comment string) - | USING clickhouse - | LOCATION '$ordersData' - | CLUSTERED BY (o_orderkey) - | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (o_orderkey, o_orderdate)"} INTO 2 BUCKETS; - |""".stripMargin) + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS orders ( + | o_orderkey bigint, + | o_custkey bigint, + | o_orderstatus string, + | o_totalprice double, + | o_orderdate date, + | o_orderpriority string, + | o_clerk string, + | o_shippriority bigint, + | o_comment string) + | USING clickhouse + | LOCATION '$ordersData' + | CLUSTERED BY (o_orderkey) + | ${if (spark32) "" else "SORTED BY (o_orderkey, o_orderdate)"} INTO 2 BUCKETS; + |""".stripMargin) val partData = tablesPath + "/part" spark.sql(s"DROP TABLE IF EXISTS part") @@ -182,24 +178,23 @@ class GlutenClickHouseTPCHBucketSuite | USING clickhouse | LOCATION '$partData' | CLUSTERED BY (p_partkey) - | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (p_partkey)"} INTO 2 BUCKETS; + | ${if (spark32) "" else "SORTED BY (p_partkey)"} INTO 2 BUCKETS; |""".stripMargin) val partsuppData = tablesPath + "/partsupp" spark.sql(s"DROP TABLE IF EXISTS partsupp") - spark.sql( - s""" - | CREATE EXTERNAL TABLE IF NOT EXISTS partsupp ( - | ps_partkey bigint, - | ps_suppkey bigint, - | ps_availqty bigint, - | ps_supplycost double, - | ps_comment string) - | USING clickhouse - | LOCATION '$partsuppData' - | CLUSTERED BY (ps_partkey) - | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (ps_partkey)"} INTO 2 BUCKETS; - |""".stripMargin) + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS partsupp ( + | ps_partkey bigint, + | ps_suppkey bigint, + | ps_availqty bigint, + | ps_supplycost double, + | ps_comment string) + | USING clickhouse + | LOCATION '$partsuppData' + | CLUSTERED BY (ps_partkey) + | ${if (spark32) "" else "SORTED BY (ps_partkey)"} INTO 2 BUCKETS; + |""".stripMargin) val supplierData = tablesPath + "/supplier" spark.sql(s"DROP TABLE IF EXISTS supplier") @@ -215,7 +210,7 @@ class GlutenClickHouseTPCHBucketSuite | USING clickhouse | LOCATION '$supplierData' | CLUSTERED BY (s_suppkey) - | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (s_suppkey)"} INTO 1 BUCKETS; + | ${if (spark32) "" else "SORTED BY (s_suppkey)"} INTO 1 BUCKETS; |""".stripMargin) val result = spark @@ -276,7 +271,7 @@ class GlutenClickHouseTPCHBucketSuite .asInstanceOf[HashJoinLikeExecTransformer] .left .isInstanceOf[InputIteratorTransformer]) - if (sparkVersion.equals("3.2")) { + if (spark32) { assert( plans(9) .asInstanceOf[HashJoinLikeExecTransformer] @@ -290,7 +285,7 @@ class GlutenClickHouseTPCHBucketSuite .isInstanceOf[FilterExecTransformerBase]) } - if (sparkVersion.equals("3.2")) { + if (spark32) { assert(!plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } else { assert(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) @@ -307,7 +302,7 @@ class GlutenClickHouseTPCHBucketSuite case scanExec: BasicScanExecTransformer => scanExec case joinExec: HashJoinLikeExecTransformer => joinExec } - if (sparkVersion.equals("3.2")) { + if (spark32) { assert( plans(1) .asInstanceOf[HashJoinLikeExecTransformer] @@ -326,7 +321,7 @@ class GlutenClickHouseTPCHBucketSuite .right .isInstanceOf[InputIteratorTransformer]) - if (sparkVersion.equals("3.2")) { + if (spark32) { assert(!plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } else { assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) @@ -344,7 +339,7 @@ class GlutenClickHouseTPCHBucketSuite ("spark.sql.optimizer.runtime.bloomFilter.enabled", "true")) { runTPCHQuery(3)( df => { - if (sparkVersion.equals("3.3")) { + if (spark33) { val plans = collectWithSubqueries(df.queryExecution.executedPlan) { case aggExec: HashAggregateExecBaseTransformer if aggExec.aggregateExpressions.exists( @@ -390,7 +385,7 @@ class GlutenClickHouseTPCHBucketSuite ("spark.sql.optimizer.runtime.bloomFilter.enabled", "true")) { runTPCHQuery(4)( df => { - if (sparkVersion.equals("3.3")) { + if (spark33) { val plans = collectWithSubqueries(df.queryExecution.executedPlan) { case aggExec: HashAggregateExecBaseTransformer if aggExec.aggregateExpressions.exists( @@ -449,7 +444,7 @@ class GlutenClickHouseTPCHBucketSuite ("spark.sql.optimizer.runtime.bloomFilter.enabled", "true")) { runTPCHQuery(12)( df => { - if (sparkVersion.equals("3.3")) { + if (spark33) { val plans = collectWithSubqueries(df.queryExecution.executedPlan) { case aggExec: HashAggregateExecBaseTransformer if aggExec.aggregateExpressions.exists( @@ -499,7 +494,7 @@ class GlutenClickHouseTPCHBucketSuite val plans = collect(df.queryExecution.executedPlan) { case joinExec: HashJoinLikeExecTransformer => joinExec } - if (sparkVersion.equals("3.2")) { + if (spark32) { assert( plans(1) .asInstanceOf[HashJoinLikeExecTransformer] @@ -546,7 +541,7 @@ class GlutenClickHouseTPCHBucketSuite ("spark.sql.optimizer.runtime.bloomFilter.enabled", "true")) { runTPCHQuery(20)( df => { - if (sparkVersion.equals("3.3")) { + if (spark33) { val plans = collectWithSubqueries(df.queryExecution.executedPlan) { case aggExec: HashAggregateExecBaseTransformer if aggExec.aggregateExpressions.exists( @@ -739,7 +734,7 @@ class GlutenClickHouseTPCHBucketSuite runSql(SQL6)( df => { checkResult(df, Array(Row(600572))) - if (sparkVersion.equals("3.2")) { + if (spark32) { // there is a shuffle between two phase hash aggregate. checkHashAggregateCount(df, 2) } else { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNotNullSkipIndexSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNotNullSkipIndexSuite.scala index 52cdaf0592ad..594d88fb6402 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNotNullSkipIndexSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNotNullSkipIndexSuite.scala @@ -32,9 +32,6 @@ class GlutenClickHouseTPCHNotNullSkipIndexSuite extends GlutenClickHouseTPCHAbst .set("spark.io.compression.codec", "SNAPPY") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") -// .set("spark.ui.enabled", "true") -// .set("spark.gluten.sql.columnar.backend.ch.runtime_config.dump_pipeline", "true") -// .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "debug") } test("test simple minmax index") { @@ -81,7 +78,7 @@ class GlutenClickHouseTPCHNotNullSkipIndexSuite extends GlutenClickHouseTPCHAbst case f: FileSourceScanExecTransformer => f } assert(scanExec.size == 1) - val mergetreeScan = scanExec(0) + val mergetreeScan = scanExec.head val ret = df.collect() assert(ret.apply(0).get(0) == 1) val marks = mergetreeScan.metrics("selectedMarks").value @@ -139,7 +136,7 @@ class GlutenClickHouseTPCHNotNullSkipIndexSuite extends GlutenClickHouseTPCHAbst case f: FileSourceScanExecTransformer => f } assert(scanExec.size == 1) - val mergetreeScan = scanExec(0) + val mergetreeScan = scanExec.head val ret = df.collect() assert(ret.apply(0).get(0) == 2) val marks = mergetreeScan.metrics("selectedMarks").value @@ -197,7 +194,7 @@ class GlutenClickHouseTPCHNotNullSkipIndexSuite extends GlutenClickHouseTPCHAbst case f: FileSourceScanExecTransformer => f } assert(scanExec.size == 1) - val mergetreeScan = scanExec(0) + val mergetreeScan = scanExec.head val ret = df.collect() assert(ret.apply(0).get(0) == 2) val marks = mergetreeScan.metrics("selectedMarks").value @@ -255,7 +252,7 @@ class GlutenClickHouseTPCHNotNullSkipIndexSuite extends GlutenClickHouseTPCHAbst case f: FileSourceScanExecTransformer => f } assert(scanExec.size == 1) - val mergetreeScan = scanExec(0) + val mergetreeScan = scanExec.head val ret = df.collect() assert(ret.apply(0).get(0) == 1) val marks = mergetreeScan.metrics("selectedMarks").value diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableSkipIndexSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableSkipIndexSuite.scala index 3d44f500a879..632cffff7cf6 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableSkipIndexSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableSkipIndexSuite.scala @@ -33,9 +33,6 @@ class GlutenClickHouseTPCHNullableSkipIndexSuite extends GlutenClickHouseTPCHAbs .set("spark.io.compression.codec", "SNAPPY") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") -// .set("spark.ui.enabled", "true") -// .set("spark.gluten.sql.columnar.backend.ch.runtime_config.dump_pipeline", "true") -// .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "debug") } test("test simple minmax index") { @@ -82,7 +79,7 @@ class GlutenClickHouseTPCHNullableSkipIndexSuite extends GlutenClickHouseTPCHAbs case f: FileSourceScanExecTransformer => f } assert(scanExec.size == 1) - val mergetreeScan = scanExec(0) + val mergetreeScan = scanExec.head val ret = df.collect() assert(ret.apply(0).get(0) == 1) val marks = mergetreeScan.metrics("selectedMarks").value @@ -140,7 +137,7 @@ class GlutenClickHouseTPCHNullableSkipIndexSuite extends GlutenClickHouseTPCHAbs case f: FileSourceScanExecTransformer => f } assert(scanExec.size == 1) - val mergetreeScan = scanExec(0) + val mergetreeScan = scanExec.head val ret = df.collect() assert(ret.apply(0).get(0) == 2) val marks = mergetreeScan.metrics("selectedMarks").value @@ -198,7 +195,7 @@ class GlutenClickHouseTPCHNullableSkipIndexSuite extends GlutenClickHouseTPCHAbs case f: FileSourceScanExecTransformer => f } assert(scanExec.size == 1) - val mergetreeScan = scanExec(0) + val mergetreeScan = scanExec.head val ret = df.collect() assert(ret.apply(0).get(0) == 2) val marks = mergetreeScan.metrics("selectedMarks").value @@ -255,7 +252,7 @@ class GlutenClickHouseTPCHNullableSkipIndexSuite extends GlutenClickHouseTPCHAbs case f: FileSourceScanExecTransformer => f } assert(scanExec.size == 1) - val mergetreeScan = scanExec(0) + val mergetreeScan = scanExec.head val ret = df.collect() assert(ret.apply(0).get(0) == 1) val marks = mergetreeScan.metrics("selectedMarks").value diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala index 6e304db196d4..4d5c6fd81956 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala @@ -41,9 +41,6 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu version(0) + "." + version(1) } - val CH_CONFIG_PREFIX: String = "spark.gluten.sql.columnar.backend.ch.runtime_config" - val CH_SETTING_PREFIX: String = "spark.gluten.sql.columnar.backend.ch.runtime_settings" - val S3_METADATA_PATH = s"/tmp/metadata/s3/$sparkVersion/" val S3_CACHE_PATH = s"/tmp/s3_cache/$sparkVersion/" val S3_ENDPOINT = "s3://127.0.0.1:9000/" @@ -53,7 +50,7 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu val HDFS_METADATA_PATH = s"/tmp/metadata/hdfs/$sparkVersion/" val HDFS_CACHE_PATH = s"/tmp/hdfs_cache/$sparkVersion/" - val HDFS_URL_ENDPOINT = s"hdfs://127.0.0.1:8020" + val HDFS_URL_ENDPOINT = "hdfs://127.0.0.1:8020" val HDFS_URL = s"$HDFS_URL_ENDPOINT/$sparkVersion" val S3_ACCESS_KEY = "BypTYzcXOlfr03FFIvt4" @@ -61,6 +58,10 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu val CH_DEFAULT_STORAGE_DIR = "/data" + protected def spark32: Boolean = sparkVersion.equals("3.2") + protected def spark33: Boolean = sparkVersion.equals("3.3") + protected def spark35: Boolean = sparkVersion.equals("3.5") + def AlmostEqualsIsRel(expected: Double, actual: Double, EPSILON: Double = DBL_EPSILON): Unit = { val diff = Math.abs(expected - actual) val epsilon = EPSILON * Math.max(Math.abs(expected), Math.abs(actual)) @@ -74,16 +75,14 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu } override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + val conf = super.sparkConf .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) - .set( - "spark.gluten.sql.columnar.backend.ch.use.v2", - ClickHouseConfig.DEFAULT_USE_DATASOURCE_V2) + .set(ClickHouseConfig.USE_DATASOURCE_V2, ClickHouseConfig.DEFAULT_USE_DATASOURCE_V2) .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.sql.warehouse.dir", warehouse) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.user_defined_path", - "/tmp/user_defined") + .setCHConfig("user_defined_path", "/tmp/user_defined") if (UTSystemParameters.testMergeTreeOnObjectStorage) { conf .set("spark.hadoop.fs.s3a.access.key", S3_ACCESS_KEY) @@ -92,100 +91,46 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu .set("spark.hadoop.fs.s3a.endpoint", MINIO_ENDPOINT) .set("spark.hadoop.fs.s3a.path.style.access", "true") .set("spark.hadoop.fs.s3a.connection.ssl.enabled", "false") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.s3.type", - "s3_gluten") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.s3.endpoint", - WHOLE_PATH) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.s3.access_key_id", - S3_ACCESS_KEY) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.s3.secret_access_key", - S3_SECRET_KEY) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.s3.metadata_path", - S3_METADATA_PATH) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.s3_cache.type", - "cache") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.s3_cache.disk", - "s3") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.s3_cache.path", - S3_CACHE_PATH) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.s3_cache.max_size", - "10Gi") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.policies.__s3_main.volumes", - "main") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.policies.__s3_main.volumes.main.disk", - "s3_cache") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs.type", - "hdfs_gluten") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs.endpoint", - HDFS_URL_ENDPOINT + "/") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs.metadata_path", - HDFS_METADATA_PATH) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache.type", - "cache") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache.disk", - "hdfs") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache.path", - HDFS_CACHE_PATH) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache.max_size", - "10Gi") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.policies.__hdfs_main.volumes", - "main") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.policies.__hdfs_main.volumes.main.disk", - "hdfs_cache") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs2.type", - "hdfs_gluten") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs2.endpoint", - HDFS_URL_ENDPOINT + "/") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs2.metadata_path", - HDFS_METADATA_PATH) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs2.metadata_type", - "rocksdb") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache2.type", - "cache") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache2.disk", - "hdfs2") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache2.path", - HDFS_CACHE_PATH) - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache2.max_size", - "10Gi") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.policies.__hdfs_main_rocksdb.volumes", - "main") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.policies.__hdfs_main_rocksdb.volumes.main.disk", - "hdfs_cache2") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.hdfs.dfs_client_read_shortcircuit", - "false") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.hdfs.dfs_default_replica", "1") + .setCHConfig( + "storage_configuration.disks.s3.type" -> "s3_gluten", + "storage_configuration.disks.s3.endpoint" -> WHOLE_PATH, + "storage_configuration.disks.s3.access_key_id" -> S3_ACCESS_KEY, + "storage_configuration.disks.s3.secret_access_key" -> S3_SECRET_KEY, + "storage_configuration.disks.s3.metadata_path" -> S3_METADATA_PATH, + "storage_configuration.disks.s3_cache.type" -> "cache", + "storage_configuration.disks.s3_cache.disk" -> "s3", + "storage_configuration.disks.s3_cache.path" -> S3_CACHE_PATH, + "storage_configuration.disks.s3_cache.max_size" -> "10Gi", + "storage_configuration.policies.__s3_main.volumes" -> "main", + "storage_configuration.policies.__s3_main.volumes.main.disk" -> "s3_cache" + ) + .setCHConfig( + "storage_configuration.disks.hdfs.type" -> "hdfs_gluten", + "storage_configuration.disks.hdfs.endpoint" -> s"$HDFS_URL_ENDPOINT/", + "storage_configuration.disks.hdfs.metadata_path" -> HDFS_METADATA_PATH, + "storage_configuration.disks.hdfs_cache.type" -> "cache", + "storage_configuration.disks.hdfs_cache.disk" -> "hdfs", + "storage_configuration.disks.hdfs_cache.path" -> HDFS_CACHE_PATH, + "storage_configuration.disks.hdfs_cache.max_size" -> "10Gi", + "storage_configuration.policies.__hdfs_main.volumes" -> "main", + "storage_configuration.policies.__hdfs_main.volumes.main.disk" -> "hdfs_cache" + ) + .setCHConfig( + "storage_configuration.disks.hdfs2.type" -> "hdfs_gluten", + "storage_configuration.disks.hdfs2.endpoint" -> s"$HDFS_URL_ENDPOINT/", + "storage_configuration.disks.hdfs2.metadata_path" -> HDFS_METADATA_PATH, + "storage_configuration.disks.hdfs2.metadata_type" -> "rocksdb", + "storage_configuration.disks.hdfs_cache2.type" -> "cache", + "storage_configuration.disks.hdfs_cache2.disk" -> "hdfs2", + "storage_configuration.disks.hdfs_cache2.path" -> HDFS_CACHE_PATH, + "storage_configuration.disks.hdfs_cache2.max_size" -> "10Gi", + "storage_configuration.policies.__hdfs_main_rocksdb.volumes" -> "main", + "storage_configuration.policies.__hdfs_main_rocksdb.volumes.main.disk" -> "hdfs_cache2" + ) + .setCHConfig( + "hdfs.dfs_client_read_shortcircuit" -> "false", + "hdfs.dfs_default_replica" -> "1" + ) } else { conf } 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 2bbbc00eb195..b268eb7192d3 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 @@ -22,6 +22,7 @@ import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, Row, TestUtils} import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, NullPropagation} +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -54,7 +55,7 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set("spark.gluten.sql.columnar.columnartorow", "true") - .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") @@ -75,9 +76,9 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS val schema = StructType( Array( - StructField("double_field1", DoubleType, true), - StructField("int_field1", IntegerType, true), - StructField("string_field1", StringType, true) + StructField("double_field1", DoubleType, nullable = true), + StructField("int_field1", IntegerType, nullable = true), + StructField("string_field1", StringType, nullable = true) )) val data = sparkContext.parallelize( Seq( @@ -103,9 +104,9 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS val dateSchema = StructType( Array( - StructField("ts", IntegerType, true), - StructField("day", DateType, true), - StructField("weekday_abbr", StringType, true) + StructField("ts", IntegerType, nullable = true), + StructField("day", DateType, nullable = true), + StructField("weekday_abbr", StringType, nullable = true) ) ) val dateRows = sparkContext.parallelize( @@ -142,7 +143,7 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS val str2MapFilePath = str2Mapfile.getAbsolutePath val str2MapSchema = StructType( Array( - StructField("str", StringType, true) + StructField("str", StringType, nullable = true) )) val str2MapData = sparkContext.parallelize( Seq( @@ -165,12 +166,12 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS .parquet(str2MapFilePath) spark.catalog.createTable("str2map_table", str2MapFilePath, fileFormat) - val urlFile = Files.createTempFile("", ".parquet").toFile() + val urlFile = Files.createTempFile("", ".parquet").toFile urlFile.deleteOnExit() val urlFilePath = urlFile.getAbsolutePath val urlTalbeSchema = StructType( Array( - StructField("url", StringType, true) + StructField("url", StringType, nullable = true) ) ) val urlTableData = sparkContext.parallelize( @@ -504,7 +505,7 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS def checkResult(df: DataFrame, exceptedResult: Seq[Row]): Unit = { // check the result val result = df.collect() - assert(result.size == exceptedResult.size) + assert(result.length === exceptedResult.size) TestUtils.compareAnswers(result, exceptedResult) } @@ -607,7 +608,7 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS test("test common subexpression eliminate") { def checkOperatorCount[T <: TransformSupport](count: Int)(df: DataFrame)(implicit tag: ClassTag[T]): Unit = { - if (sparkVersion.equals("3.3")) { + if (spark33) { assert( getExecutedPlan(df).count( plan => { @@ -723,10 +724,10 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS val transform_sql = "select transform(ids, x -> x + 1) from tb_array" runQueryAndCompare(transform_sql)(checkGlutenOperatorMatch[ProjectExecTransformer]) - val filter_sql = "select filter(ids, x -> x % 2 == 1) from tb_array"; + val filter_sql = "select filter(ids, x -> x % 2 == 1) from tb_array" runQueryAndCompare(filter_sql)(checkGlutenOperatorMatch[ProjectExecTransformer]) - val aggregate_sql = "select ids, aggregate(ids, 3, (acc, x) -> acc + x) from tb_array"; + val aggregate_sql = "select ids, aggregate(ids, 3, (acc, x) -> acc + x) from tb_array" runQueryAndCompare(aggregate_sql)(checkGlutenOperatorMatch[ProjectExecTransformer]) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala index 9c3dbcac3245..f16c897671b1 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala @@ -21,6 +21,7 @@ import org.apache.gluten.execution.GlutenClickHouseTPCHAbstractSuite import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { override protected val needCopyParquetToTablePath = true @@ -50,7 +51,7 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set("spark.gluten.sql.columnar.columnartorow", "true") - .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala index 6778dccd3340..4e2b5ad63e0a 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, SaveMode} import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.hive.HiveTableScanExecTransformer import org.apache.spark.sql.internal.SQLConf @@ -40,6 +41,8 @@ class GlutenClickHouseHiveTableSuite with AdaptiveSparkPlanHelper { override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + new SparkConf() .set("spark.plugins", "org.apache.gluten.GlutenPlugin") .set("spark.memory.offHeap.enabled", "true") @@ -52,7 +55,7 @@ class GlutenClickHouseHiveTableSuite .set("spark.sql.adaptive.enabled", "false") .set("spark.sql.files.minPartitionNum", "1") .set("spark.gluten.sql.columnar.columnartorow", "true") - .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") @@ -60,10 +63,10 @@ class GlutenClickHouseHiveTableSuite .set("spark.gluten.sql.parquet.maxmin.index", "true") .set( "spark.sql.warehouse.dir", - getClass.getResource("/").getPath + "tests-working-home/spark-warehouse") + this.getClass.getResource("/").getPath + "tests-working-home/spark-warehouse") .set("spark.hive.exec.dynamic.partition.mode", "nonstrict") .set("spark.gluten.supported.hive.udfs", "my_add") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format", "true") + .setCHConfig("use_local_format", true) .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") .set( "spark.sql.catalog.spark_catalog", @@ -1009,7 +1012,7 @@ class GlutenClickHouseHiveTableSuite def checkOperatorCount[T <: TransformSupport](count: Int)(df: DataFrame)(implicit tag: ClassTag[T]): Unit = { - if (sparkVersion.equals("3.3")) { + if (spark33) { assert( getExecutedPlan(df).count( plan => { @@ -1350,7 +1353,7 @@ class GlutenClickHouseHiveTableSuite sql(insertSql) val selectSql = s"SELECT * FROM $tableName" - compareResultsAgainstVanillaSpark(selectSql, true, _ => {}) + compareResultsAgainstVanillaSpark(selectSql, compareResult = true, _ => {}) sql(s"drop table if exists $tableName") } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala index 9e3fa00787de..53aef16d143e 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.SparkConf import org.apache.spark.gluten.NativeWriteChecker import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.types._ import scala.reflect.runtime.universe.TypeTag @@ -36,6 +37,8 @@ class GlutenClickHouseNativeWriteTableSuite with NativeWriteChecker { override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + var sessionTimeZone = "GMT" if (isSparkVersionGE("3.5")) { sessionTimeZone = java.util.TimeZone.getDefault.getID @@ -55,7 +58,7 @@ class GlutenClickHouseNativeWriteTableSuite .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set("spark.gluten.sql.columnar.columnartorow", "true") - .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") @@ -64,7 +67,7 @@ class GlutenClickHouseNativeWriteTableSuite .set("spark.sql.storeAssignmentPolicy", "legacy") .set("spark.sql.warehouse.dir", getWarehouseDir) .set("spark.sql.session.timeZone", sessionTimeZone) - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") + .setCHConfig("logger.level", "error") .setMaster("local[1]") } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala index d359428d03ca..503c9bd886ed 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala @@ -46,27 +46,21 @@ class GlutenClickHouseTableAfterRestart /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.user_defined_path", - "/tmp/user_defined") + .setCHConfig("logger.level", "error") + .setCHConfig("user_defined_path", "/tmp/user_defined") .set("spark.sql.files.maxPartitionBytes", "20000000") .set("spark.ui.enabled", "true") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.min_insert_block_size_rows", - "100000") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", - "false") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.input_format_parquet_max_block_size", - "8192") + .setCHSettings("min_insert_block_size_rows", 100000) + .setCHSettings("mergetree.merge_after_insert", false) + .setCHSettings("input_format_parquet_max_block_size", 8192) .setMaster("local[2]") } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeCacheDataSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala similarity index 97% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeCacheDataSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala index 73dc5879ea6c..decd27099ea7 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeCacheDataSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala @@ -14,7 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.mergetree + +import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseTPCHAbstractSuite} import org.apache.spark.SparkConf import org.apache.spark.sql.delta.files.TahoeFileIndex @@ -29,9 +32,6 @@ import java.io.File import scala.concurrent.duration.DurationInt -// Some sqls' line length exceeds 100 -// scalastyle:off line.size.limit - class GlutenClickHouseMergeTreeCacheDataSuite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { @@ -47,18 +47,18 @@ class GlutenClickHouseMergeTreeCacheDataSuite } override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") + .setCHConfig("logger.level", "error") .set("spark.gluten.soft-affinity.enabled", "true") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", - "false") - // .set("spark.gluten.sql.columnar.backend.ch.runtime_config.path", "/data") // for local test + .setCHSettings("mergetree.merge_after_insert", false) + .setCHConfig("path", "/data") } override protected def beforeEach(): Unit = { @@ -593,7 +593,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite } test("test disable cache files return") { - withSQLConf(s"$CH_CONFIG_PREFIX.gluten_cache.local.enabled" -> "false") { + withSQLConf(CHConf.runtimeConfig("gluten_cache.local.enabled") -> "false") { runSql( s"CACHE FILES select * from '$HDFS_URL_ENDPOINT/tpch-data/lineitem'", noFallBack = false) { @@ -605,4 +605,3 @@ class GlutenClickHouseMergeTreeCacheDataSuite } } } -// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeOptimizeSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeOptimizeSuite.scala similarity index 92% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeOptimizeSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeOptimizeSuite.scala index 7989c02ba872..9effd64a277f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeOptimizeSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeOptimizeSuite.scala @@ -14,7 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.mergetree + +import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseTPCHAbstractSuite} import org.apache.spark.SparkConf import org.apache.spark.sql.SaveMode @@ -26,9 +29,6 @@ import java.io.File import scala.concurrent.duration.DurationInt -// Some sqls' line length exceeds 100 -// scalastyle:off line.size.limit - class GlutenClickHouseMergeTreeOptimizeSuite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { @@ -41,27 +41,22 @@ class GlutenClickHouseMergeTreeOptimizeSuite /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.min_insert_block_size_rows", - "10000" - ) + .setCHConfig("logger.level", "error") + .setCHSettings("min_insert_block_size_rows", 10000) .set( "spark.databricks.delta.retentionDurationCheck.enabled", "false" - ) // otherwise RETAIN 0 HOURS will fail - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", - "false") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.input_format_parquet_max_block_size", - "8192") + ) // otherwise, RETAIN 0 HOURS will fail + .setCHSettings("mergetree.merge_after_insert", false) + .setCHSettings("input_format_parquet_max_block_size", 8192) } override protected def createTPCHNotNullTables(): Unit = { @@ -119,7 +114,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sparkContext.setJobGroup("test", "test") spark.sql("optimize lineitem_mergetree_optimize_p") val job_ids = spark.sparkContext.statusTracker.getJobIdsForGroup("test") - if (sparkVersion.equals("3.5")) { + if (spark35) { assertResult(4)(job_ids.length) } else { assertResult(1)(job_ids.length) // will not trigger actual merge job @@ -131,7 +126,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite assertResult(22728)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p"))) spark.sql("VACUUM lineitem_mergetree_optimize_p RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(22728)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p"))) } else { // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. @@ -158,7 +153,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sparkContext.setJobGroup("test2", "test2") spark.sql("optimize lineitem_mergetree_optimize_p2") val job_ids = spark.sparkContext.statusTracker.getJobIdsForGroup("test2") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(7)(job_ids.length) // WILL trigger actual merge job } else { assertResult(8)(job_ids.length) // WILL trigger actual merge job @@ -171,14 +166,14 @@ class GlutenClickHouseMergeTreeOptimizeSuite assertResult(372)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2"))) spark.sql("VACUUM lineitem_mergetree_optimize_p2 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(239)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2"))) } else { assertResult(241)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2"))) } spark.sql("VACUUM lineitem_mergetree_optimize_p2 RETAIN 0 HOURS") // the second VACUUM will remove some empty folders - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(220)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2"))) } else { assertResult(226)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2"))) @@ -208,13 +203,13 @@ class GlutenClickHouseMergeTreeOptimizeSuite assertResult(516)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3"))) spark.sql("VACUUM lineitem_mergetree_optimize_p3 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(306)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3"))) } else { assertResult(308)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3"))) } spark.sql("VACUUM lineitem_mergetree_optimize_p3 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(276)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3"))) } else { assertResult(282)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3"))) @@ -245,13 +240,13 @@ class GlutenClickHouseMergeTreeOptimizeSuite assertResult(516)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4"))) spark.sql("VACUUM lineitem_mergetree_optimize_p4 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(306)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4"))) } else { assertResult(308)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4"))) } spark.sql("VACUUM lineitem_mergetree_optimize_p4 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(276)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4"))) } else { assertResult(282)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4"))) @@ -281,7 +276,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(99)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5"))) } else { // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. @@ -305,7 +300,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(93)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5"))) } else { // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. @@ -321,7 +316,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(77)(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5"))) } else { // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. @@ -343,7 +338,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite |PARTITIONED BY (l_returnflag) |LOCATION '$basePath/lineitem_mergetree_optimize_p6' | CLUSTERED BY (l_partkey) - | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_partkey)"} INTO 2 BUCKETS + | ${if (spark32) "" else "SORTED BY (l_partkey)"} INTO 2 BUCKETS | as select * from lineitem |""".stripMargin) @@ -353,11 +348,11 @@ class GlutenClickHouseMergeTreeOptimizeSuite assertResult(600572)(ret.apply(0).get(0)) assertResult(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p6")))( - if (sparkVersion.equals("3.2")) 499 else 528) + if (spark32) 499 else 528) 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"$basePath/lineitem_mergetree_optimize_p6")))( - if (sparkVersion.equals("3.2")) 315 else 327) + if (spark32) 315 else 327) val ret2 = spark.sql("select count(*) from lineitem_mergetree_optimize_p6").collect() assertResult(600572)(ret2.apply(0).get(0)) @@ -426,7 +421,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite clickhouseTable.vacuum(0.0) clickhouseTable.vacuum(0.0) - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(99)(countFiles(new File(dataPath))) } else { assertResult(105)(countFiles(new File(dataPath))) @@ -449,7 +444,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite clickhouseTable.vacuum(0.0) clickhouseTable.vacuum(0.0) - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(93)(countFiles(new File(dataPath))) } else { assertResult(104)(countFiles(new File(dataPath))) @@ -465,7 +460,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite clickhouseTable.vacuum(0.0) clickhouseTable.vacuum(0.0) - if (sparkVersion.equals("3.2")) { + if (spark32) { assertResult(77)(countFiles(new File(dataPath))) } else { assertResult(93)(countFiles(new File(dataPath))) @@ -478,7 +473,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite test("test mergetree insert with optimize basic") { withSQLConf( "spark.databricks.delta.optimize.minFileSize" -> "200000000", - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert" -> "true" + CHConf.runtimeSettings("mergetree.merge_after_insert") -> "true" ) { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_mergetree_insert_optimize_basic; @@ -501,4 +496,3 @@ class GlutenClickHouseMergeTreeOptimizeSuite } } } -// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala similarity index 96% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala index 34ffecb45833..0a8860aaa57d 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.mergetree + +import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.SaveMode @@ -32,9 +34,6 @@ import java.io.File import scala.io.Source -// Some sqls' line length exceeds 100 -// scalastyle:off line.size.limit - class GlutenClickHouseMergeTreePathBasedWriteSuite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { @@ -47,6 +46,8 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") @@ -55,15 +56,10 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .set("spark.sql.adaptive.enabled", "true") .set("spark.sql.files.maxPartitionBytes", "20000000") .set("spark.ui.enabled", "true") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.min_insert_block_size_rows", - "100000") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", - "false") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.input_format_parquet_max_block_size", - "8192") + .setCHSettings("min_insert_block_size_rows", 100000) + .setCHSettings("mergetree.merge_after_insert", false) + .setCHSettings("input_format_parquet_max_block_size", 8192) + } override protected def createTPCHNotNullTables(): Unit = { @@ -509,17 +505,23 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite merge into clickhouse.`$dataPath` using ( - select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, - 'Z' as `l_returnflag`, - l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment - from lineitem where l_orderkey in (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) and l_orderkey < 100000 + select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, + l_extendedprice, l_discount, l_tax, 'Z' as `l_returnflag`, + l_linestatus, l_shipdate, l_commitdate, l_receiptdate, + l_shipinstruct, l_shipmode, l_comment + from lineitem where l_orderkey in + (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) + and l_orderkey < 100000 union select l_orderkey + 10000000, - l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, - l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment - from lineitem where l_orderkey in (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) and l_orderkey < 100000 + l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, + l_discount, l_tax, l_returnflag,l_linestatus, l_shipdate, l_commitdate, + l_receiptdate, l_shipinstruct, l_shipmode, l_comment + from lineitem where l_orderkey in + (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) + and l_orderkey < 100000 ) as updates on updates.l_orderkey = clickhouse.`$dataPath`.l_orderkey @@ -922,15 +924,14 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite val dataPath = s"$basePath/lineitem_mergetree_ctas2" clearDataPath(dataPath) - spark.sql( - s""" - |CREATE TABLE clickhouse.`$dataPath` - |USING clickhouse - |PARTITIONED BY (l_shipdate) - |CLUSTERED BY (l_orderkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_partkey, l_returnflag)"} INTO 4 BUCKETS - | as select * from lineitem - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE clickhouse.`$dataPath` + |USING clickhouse + |PARTITIONED BY (l_shipdate) + |CLUSTERED BY (l_orderkey) + |${if (spark32) "" else "SORTED BY (l_partkey, l_returnflag)"} INTO 4 BUCKETS + | as select * from lineitem + |""".stripMargin) val sqlStr = s""" @@ -1332,9 +1333,8 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite val result = df.collect() assertResult(600572)(result(0).getLong(0)) // Spark 3.2 + Delta 2.0 does not support this feature - if (!sparkVersion.equals("3.2")) { + if (!spark32) { assert(df.queryExecution.executedPlan.isInstanceOf[LocalTableScanExec]) } } } -// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala similarity index 96% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala index bbfac80a7374..83990d68ad47 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala @@ -14,7 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.mergetree + +import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseTPCHAbstractSuite} import org.apache.spark.SparkConf import org.apache.spark.sql.SaveMode @@ -31,9 +34,6 @@ import java.io.File import scala.concurrent.duration.DurationInt -// Some sqls' line length exceeds 100 -// scalastyle:off line.size.limit - class GlutenClickHouseMergeTreeWriteOnHDFSSuite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { @@ -49,17 +49,17 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite } override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", - "false") - // .set("spark.gluten.sql.columnar.backend.ch.runtime_config.path", "/data") // for local test + .setCHConfig("logger.level", "error") + .setCHSettings("mergetree.merge_after_insert", false) + .setCHConfig("path", "/data") } override protected def beforeEach(): Unit = { @@ -462,7 +462,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite |USING clickhouse |PARTITIONED BY (l_returnflag) |CLUSTERED BY (l_orderkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_partkey)"} INTO 4 BUCKETS + |${if (spark32) "" else "SORTED BY (l_partkey)"} INTO 4 BUCKETS |LOCATION '$HDFS_URL/test/lineitem_mergetree_bucket_hdfs' |TBLPROPERTIES (storage_policy='__hdfs_main') |""".stripMargin) @@ -510,7 +510,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).bucketOption.isDefined) - if (sparkVersion.equals("3.2")) { + if (spark32) { assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).orderByKeyOption.isEmpty) } else { assertResult("l_partkey")( @@ -623,9 +623,9 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite withSQLConf( "spark.databricks.delta.optimize.minFileSize" -> "200000000", - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert" -> "true", - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.insert_without_local_storage" -> "true", - "spark.gluten.sql.columnar.backend.ch.runtime_settings.min_insert_block_size_rows" -> "10000" + CHConf.runtimeSettings("mergetree.merge_after_insert") -> "true", + CHConf.runtimeSettings("mergetree.insert_without_local_storage") -> "true", + CHConf.runtimeSettings("min_insert_block_size_rows") -> "10000" ) { spark.sql(s""" |DROP TABLE IF EXISTS $tableName; @@ -657,4 +657,3 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite } } } -// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala similarity index 96% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala index 6454f155bf09..3830f8775aaa 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala @@ -14,7 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.mergetree + +import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseTPCHAbstractSuite} import org.apache.spark.SparkConf import org.apache.spark.sql.SaveMode @@ -31,9 +34,6 @@ import java.io.File import scala.concurrent.duration.DurationInt -// Some sqls' line length exceeds 100 -// scalastyle:off line.size.limit - class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { @@ -49,17 +49,17 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite } override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", - "false") - + .setCHConfig("logger.level", "error") + .setCHSettings("mergetree.merge_after_insert", false) + .setCHConfig("path", "/data") } override protected def beforeEach(): Unit = { @@ -462,7 +462,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite |USING clickhouse |PARTITIONED BY (l_returnflag) |CLUSTERED BY (l_orderkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_partkey)"} INTO 4 BUCKETS + |${if (spark32) "" else "SORTED BY (l_partkey)"} INTO 4 BUCKETS |LOCATION '$HDFS_URL/test/lineitem_mergetree_bucket_hdfs' |TBLPROPERTIES (storage_policy='__hdfs_main_rocksdb') |""".stripMargin) @@ -510,7 +510,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).bucketOption.isDefined) - if (sparkVersion.equals("3.2")) { + if (spark32) { assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).orderByKeyOption.isEmpty) } else { assertResult("l_partkey")( @@ -623,9 +623,9 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite withSQLConf( "spark.databricks.delta.optimize.minFileSize" -> "200000000", - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert" -> "true", - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.insert_without_local_storage" -> "true", - "spark.gluten.sql.columnar.backend.ch.runtime_settings.min_insert_block_size_rows" -> "10000" + CHConf.runtimeSettings("mergetree.merge_after_insert") -> "true", + CHConf.runtimeSettings("mergetree.insert_without_local_storage") -> "true", + CHConf.runtimeSettings("min_insert_block_size_rows") -> "10000" ) { spark.sql(s""" |DROP TABLE IF EXISTS $tableName; @@ -657,4 +657,3 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite } } } -// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnS3Suite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala similarity index 96% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnS3Suite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala index 87e95cbe9dda..13d3a53d3031 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnS3Suite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala @@ -14,17 +14,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.mergetree + +import org.apache.gluten.backendsapi.clickhouse.CHConf._ +import org.apache.gluten.execution.{BasicScanExecTransformer, FileSourceScanExecTransformer, GlutenClickHouseTPCHAbstractSuite} import org.apache.spark.SparkConf +import org.apache.spark.sql.SaveMode import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 import org.apache.spark.sql.delta.files.TahoeFileIndex +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata.AddMergeTreeParts import _root_.org.apache.commons.io.FileUtils -import _root_.org.apache.spark.sql.SaveMode -import _root_.org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import io.minio.{BucketExistsArgs, ListObjectsArgs, MakeBucketArgs, MinioClient, RemoveBucketArgs, RemoveObjectsArgs} +import io.minio._ import io.minio.messages.DeleteObject import java.io.File @@ -32,9 +35,6 @@ import java.util import scala.concurrent.duration.DurationInt -// Some sqls' line length exceeds 100 -// scalastyle:off line.size.limit - class GlutenClickHouseMergeTreeWriteOnS3Suite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { @@ -62,8 +62,8 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") - // .set("spark.gluten.sql.columnar.backend.ch.runtime_config.path", "/data") // for local test + .setCHConfig("logger.level", "error") + .setCHConfig("path", "/data") } override protected def beforeEach(): Unit = { @@ -523,7 +523,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite |USING clickhouse |PARTITIONED BY (l_returnflag) |CLUSTERED BY (l_orderkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_partkey)"} INTO 4 BUCKETS + |${if (spark32) "" else "SORTED BY (l_partkey)"} INTO 4 BUCKETS |LOCATION 's3a://$BUCKET_NAME/lineitem_mergetree_bucket_s3' |TBLPROPERTIES (storage_policy='__s3_main') |""".stripMargin) @@ -571,7 +571,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).bucketOption.isDefined) - if (sparkVersion.equals("3.2")) { + if (spark32) { assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).orderByKeyOption.isEmpty) } else { assertResult("l_partkey")( @@ -684,8 +684,8 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite withSQLConf( "spark.databricks.delta.optimize.minFileSize" -> "200000000", - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.insert_without_local_storage" -> "true", - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert" -> "true" + runtimeSettings("mergetree.insert_without_local_storage") -> "true", + runtimeSettings("mergetree.merge_after_insert") -> "true" ) { spark.sql(s""" |DROP TABLE IF EXISTS $tableName; @@ -757,8 +757,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite | AND l_quantity < 24 |""".stripMargin - withSQLConf( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.enabled_driver_filter_mergetree_index" -> "true") { + withSQLConf(runtimeSettings("enabled_driver_filter_mergetree_index") -> "true") { runTPCHQueryBySQL(6, sqlStr) { df => val scanExec = collect(df.queryExecution.executedPlan) { @@ -806,7 +805,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite |USING clickhouse |PARTITIONED BY (l_returnflag) |CLUSTERED BY (l_orderkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_partkey)"} INTO 4 BUCKETS + |${if (spark32) "" else "SORTED BY (l_partkey)"} INTO 4 BUCKETS |LOCATION 's3a://$BUCKET_NAME/lineitem_mergetree_bucket_s3' |TBLPROPERTIES (storage_policy='__s3_main') |""".stripMargin) @@ -821,4 +820,3 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite spark.sql("drop table lineitem_mergetree_bucket_s3") } } -// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala similarity index 95% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index 3b7606daac6b..648198590b2c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -14,7 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.mergetree + +import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.execution._ +import org.apache.gluten.utils.Arm import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, SaveMode} @@ -30,9 +34,6 @@ import java.io.File import scala.io.Source -// Some sqls' line length exceeds 100 -// scalastyle:off line.size.limit - class GlutenClickHouseMergeTreeWriteSuite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { @@ -43,6 +44,8 @@ class GlutenClickHouseMergeTreeWriteSuite override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" override protected val queriesResults: String = rootPath + "mergetree-queries-output" + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { super.sparkConf @@ -52,15 +55,9 @@ class GlutenClickHouseMergeTreeWriteSuite .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") .set("spark.sql.files.maxPartitionBytes", "20000000") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.min_insert_block_size_rows", - "100000") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", - "false") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.input_format_parquet_max_block_size", - "8192") + .setCHSettings("min_insert_block_size_rows", 100000) + .setCHSettings("mergetree.merge_after_insert", false) + .setCHSettings("input_format_parquet_max_block_size", 8192) } override protected def createTPCHNotNullTables(): Unit = { @@ -152,7 +149,7 @@ class GlutenClickHouseMergeTreeWriteSuite val planNodeJson = wholeStageTransformer.substraitPlanJson assert( !planNodeJson - .replaceAll("\\\n", "") + .replaceAll("\n", "") .replaceAll(" ", "") .contains("\"input\":{\"filter\":{")) } @@ -485,18 +482,23 @@ class GlutenClickHouseMergeTreeWriteSuite merge into $tableName using ( - select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, - 'Z' as `l_returnflag`, - l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment - from lineitem where l_orderkey in (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) and l_orderkey < 100000 + select l_orderkey, l_partkey, l_suppkey, l_linenumber, + l_quantity, l_extendedprice, l_discount, l_tax, + 'Z' as `l_returnflag`,l_linestatus, l_shipdate, l_commitdate, + l_receiptdate, l_shipinstruct, l_shipmode, l_comment + from lineitem where l_orderkey in + (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) + and l_orderkey < 100000 union select l_orderkey + 10000000, - l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, - l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment - from lineitem where l_orderkey in (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) and l_orderkey < 100000 - + l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, + l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, + l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment + from lineitem where l_orderkey in + (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) + and l_orderkey < 100000 ) as updates on updates.l_orderkey = $tableName.l_orderkey when matched then update set * @@ -829,7 +831,7 @@ class GlutenClickHouseMergeTreeWriteSuite |USING clickhouse |PARTITIONED BY (l_returnflag) |CLUSTERED BY (l_partkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_orderkey)"} INTO 4 BUCKETS + |${if (spark32) "" else "SORTED BY (l_orderkey)"} INTO 4 BUCKETS |LOCATION '$basePath/lineitem_mergetree_bucket' |""".stripMargin) @@ -876,7 +878,7 @@ class GlutenClickHouseMergeTreeWriteSuite val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).bucketOption.isDefined) - if (sparkVersion.equals("3.2")) { + if (spark32) { assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).orderByKeyOption.isEmpty) } else { assertResult("l_orderkey")( @@ -1096,16 +1098,15 @@ class GlutenClickHouseMergeTreeWriteSuite |DROP TABLE IF EXISTS lineitem_mergetree_ctas2; |""".stripMargin) - spark.sql( - s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree_ctas2 - |USING clickhouse - |PARTITIONED BY (l_shipdate) - |CLUSTERED BY (l_orderkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_partkey, l_returnflag)"} INTO 4 BUCKETS - |LOCATION '$basePath/lineitem_mergetree_ctas2' - | as select * from lineitem - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_ctas2 + |USING clickhouse + |PARTITIONED BY (l_shipdate) + |CLUSTERED BY (l_orderkey) + |${if (spark32) "" else "SORTED BY (l_partkey, l_returnflag)"} INTO 4 BUCKETS + |LOCATION '$basePath/lineitem_mergetree_ctas2' + | as select * from lineitem + |""".stripMargin) val sqlStr = s""" @@ -1201,7 +1202,7 @@ class GlutenClickHouseMergeTreeWriteSuite // find a folder whose name is like 48b70783-b3b8-4bf8-9c52-5261aead8e3e_0_006 val partDir = directory.listFiles().filter(f => f.getName.length > 20).head val columnsFile = new File(partDir, "columns.txt") - val columns = Source.fromFile(columnsFile).getLines().mkString + val columns = Arm.withResource(Source.fromFile(columnsFile))(_.getLines().mkString) assert(columns.contains("`l_returnflag` LowCardinality(Nullable(String))")) assert(columns.contains("`l_linestatus` LowCardinality(Nullable(String))")) @@ -1782,8 +1783,7 @@ class GlutenClickHouseMergeTreeWriteSuite Seq(("true", 2), ("false", 3)).foreach( conf => { - withSQLConf( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.enabled_driver_filter_mergetree_index" -> conf._1) { + withSQLConf(CHConf.runtimeSettings("enabled_driver_filter_mergetree_index") -> conf._1) { runTPCHQueryBySQL(6, sqlStr) { df => val scanExec = collect(df.queryExecution.executedPlan) { @@ -1812,32 +1812,31 @@ class GlutenClickHouseMergeTreeWriteSuite |DROP TABLE IF EXISTS orders_mergetree_pk_pruning_by_driver_bucket; |""".stripMargin) - spark.sql( - s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree_pk_pruning_by_driver_bucket - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |CLUSTERED by (l_orderkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_receiptdate)"} INTO 2 BUCKETS - |LOCATION '$basePath/lineitem_mergetree_pk_pruning_by_driver_bucket' - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_pk_pruning_by_driver_bucket + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |CLUSTERED by (l_orderkey) + |${if (spark32) "" else "SORTED BY (l_receiptdate)"} INTO 2 BUCKETS + |LOCATION '$basePath/lineitem_mergetree_pk_pruning_by_driver_bucket' + |""".stripMargin) spark.sql(s""" |CREATE TABLE IF NOT EXISTS orders_mergetree_pk_pruning_by_driver_bucket ( @@ -1852,7 +1851,7 @@ class GlutenClickHouseMergeTreeWriteSuite | o_comment string) |USING clickhouse |CLUSTERED by (o_orderkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (o_orderdate)"} INTO 2 BUCKETS + |${if (spark32) "" else "SORTED BY (o_orderdate)"} INTO 2 BUCKETS |LOCATION '$basePath/orders_mergetree_pk_pruning_by_driver_bucket' |""".stripMargin) @@ -1891,7 +1890,8 @@ class GlutenClickHouseMergeTreeWriteSuite | AND l_shipmode IN ('MAIL', 'SHIP') | AND l_commitdate < l_receiptdate | AND l_shipdate < l_commitdate - | AND l_receiptdate >= date'1994-01-01' AND l_receiptdate < date'1994-01-01' + interval 1 year + | AND l_receiptdate >= date'1994-01-01' + | AND l_receiptdate < date'1994-01-01' + interval 1 year |GROUP BY | l_shipmode |ORDER BY @@ -1900,8 +1900,7 @@ class GlutenClickHouseMergeTreeWriteSuite Seq(("true", 2), ("false", 2)).foreach( conf => { - withSQLConf( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.enabled_driver_filter_mergetree_index" -> conf._1) { + withSQLConf(CHConf.runtimeSettings("enabled_driver_filter_mergetree_index") -> conf._1) { runTPCHQueryBySQL(12, sqlStr) { df => val scanExec = collect(df.queryExecution.executedPlan) { @@ -1966,7 +1965,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult("600572")(result(0).getLong(0).toString) // Spark 3.2 + Delta 2.0 does not support this feature - if (!sparkVersion.equals("3.2")) { + if (!spark32) { assert(df.queryExecution.executedPlan.isInstanceOf[LocalTableScanExec]) } }) @@ -2051,4 +2050,3 @@ class GlutenClickHouseMergeTreeWriteSuite runSql(sqlStr) { _ => } } } -// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala similarity index 93% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala index dded88b02bce..5f03a0d39f50 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala @@ -14,16 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.mergetree -import org.apache.gluten.backendsapi.clickhouse.CHBackend +import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.execution.GlutenClickHouseTPCHAbstractSuite import org.apache.spark.SparkConf import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -// Some sqls' line length exceeds 100 -// scalastyle:off line.size.limit - class GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { @@ -52,8 +50,7 @@ class GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite test("GLUTEN-6470: Fix Task not serializable error when inserting mergetree data") { - val externalSortKey = s"${CHBackend.CONF_PREFIX}.runtime_settings" + - s".max_bytes_before_external_sort" + val externalSortKey = CHConf.runtimeSettings("max_bytes_before_external_sort") assertResult(3435973836L)(spark.conf.get(externalSortKey).toLong) spark.sql(s""" @@ -117,4 +114,3 @@ class GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite runTPCHQueryBySQL(1, sqlStr)(_ => {}) } } -// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickhouseMergetreeSoftAffinitySuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickhouseMergetreeSoftAffinitySuite.scala similarity index 96% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickhouseMergetreeSoftAffinitySuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickhouseMergetreeSoftAffinitySuite.scala index d5620f5df4c1..97418c670f71 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickhouseMergetreeSoftAffinitySuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickhouseMergetreeSoftAffinitySuite.scala @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.mergetree import org.apache.gluten.affinity.{CHUTAffinity, CHUTSoftAffinityManager} +import org.apache.gluten.execution.{GlutenClickHouseTPCHAbstractSuite, GlutenMergeTreePartition, MergeTreePartSplit} import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala index a7ec27d82f08..24b50dfebe7c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala @@ -41,18 +41,16 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite // scalastyle:off line.size.limit /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "1") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - // .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "DEBUG") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_settings.input_format_parquet_max_block_size", - s"$parquetMaxBlockSize") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.enable_streaming_aggregating", - "true") + .setCHConfig("logger.level", "error") + .setCHSettings("input_format_parquet_max_block_size", parquetMaxBlockSize) + .setCHConfig("enable_streaming_aggregating", true) } // scalastyle:on line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetColumnIndexSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetColumnIndexSuite.scala index 0311594a18ad..01d01d3e85f4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetColumnIndexSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetColumnIndexSuite.scala @@ -96,8 +96,11 @@ class GlutenParquetColumnIndexSuite val chFileScan = chScanPlan.head assertResult(scanOutput)(chFileScan.longMetric("numOutputRows").value) } - override protected def sparkConf: SparkConf = + override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format", "true") + .setCHConfig("use_local_format", true) + } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala index 4b4a0d34e89a..cc0bfd9cedea 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala @@ -26,14 +26,16 @@ class GlutenClickHouseTPCDSParquetGraceHashJoinSuite extends GlutenClickHouseTPC /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "sort") .set("spark.io.compression.codec", "snappy") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.memory.offHeap.size", "6g") - .set("spark.gluten.sql.columnar.backend.ch.runtime_settings.join_algorithm", "grace_hash") - .set("spark.gluten.sql.columnar.backend.ch.runtime_settings.max_bytes_in_join", "314572800") + .setCHSettings("join_algorithm", "grace_hash") + .setCHSettings("max_bytes_in_join", 314572800) .setMaster("local[2]") } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala index fbea2ed464d7..614780dbbded 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.execution.tpch +import org.apache.gluten.backendsapi.clickhouse.CHConf._ import org.apache.gluten.execution.{CHNativeCacheManager, FileSourceScanExecTransformer, GlutenClickHouseTPCHAbstractSuite} import org.apache.spark.SparkConf @@ -43,14 +44,17 @@ class GlutenClickHouseHDFSSuite .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") - .set(s"$CH_CONFIG_PREFIX.use_local_format", "true") - .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") - .set(s"$CH_CONFIG_PREFIX.gluten_cache.local.enabled", "true") - .set(s"$CH_CONFIG_PREFIX.gluten_cache.local.name", cache_name) - .set(s"$CH_CONFIG_PREFIX.gluten_cache.local.path", hdfsCachePath) - .set(s"$CH_CONFIG_PREFIX.gluten_cache.local.max_size", "10Gi") - .set(s"$CH_CONFIG_PREFIX.reuse_disk_cache", "false") + .setCHConfig("use_local_format", true) + .set(prefixOf("shuffle.hash.algorithm"), "sparkMurmurHash3_32") + .setCHConfig("gluten_cache.local.enabled", "true") + .setCHConfig("gluten_cache.local.name", cache_name) + .setCHConfig("gluten_cache.local.path", hdfsCachePath) + .setCHConfig("gluten_cache.local.max_size", "10Gi") + .setCHConfig("reuse_disk_cache", "false") .set("spark.sql.adaptive.enabled", "false") + + // TODO: spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm => + // CHConf.prefixOf("shuffle.hash.algorithm") } override protected def createTPCHNotNullTables(): Unit = { @@ -123,7 +127,7 @@ class GlutenClickHouseHDFSSuite ignore("test no cache by query") { withSQLConf( - s"$CH_SETTING_PREFIX.read_from_filesystem_cache_if_exists_otherwise_bypass_cache" -> "true") { + runtimeSettings("read_from_filesystem_cache_if_exists_otherwise_bypass_cache") -> "true") { runWithoutCache() } 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 28b874e21bac..1ca9ab7f45b2 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 @@ -38,6 +38,8 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") @@ -45,9 +47,7 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.enable_streaming_aggregating", - "true") + .setCHConfig("enable_streaming_aggregating", true) } override protected def createTPCHNotNullTables(): Unit = { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala index 5f9f01fb697a..d06cd42a8d60 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala @@ -39,13 +39,15 @@ class GlutenClickHouseTPCHParquetAQESuite /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ + super.sparkConf .set("spark.shuffle.manager", "sort") .set("spark.io.compression.codec", "snappy") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format", "true") + .setCHConfig("use_local_format", true) .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") } 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 614e0124b9ff..5e20293ded9d 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 @@ -306,7 +306,7 @@ class GlutenClickHouseTPCHParquetBucketSuite .asInstanceOf[HashJoinLikeExecTransformer] .left .isInstanceOf[InputIteratorTransformer]) - if (sparkVersion.equals("3.2")) { + if (spark32) { assert( plans(9) .asInstanceOf[HashJoinLikeExecTransformer] @@ -320,7 +320,7 @@ class GlutenClickHouseTPCHParquetBucketSuite .isInstanceOf[FilterExecTransformerBase]) } - if (sparkVersion.equals("3.2")) { + if (spark32) { assert(!plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } else { assert(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) @@ -340,7 +340,7 @@ class GlutenClickHouseTPCHParquetBucketSuite case scanExec: BasicScanExecTransformer => scanExec case joinExec: HashJoinLikeExecTransformer => joinExec } - if (sparkVersion.equals("3.2")) { + if (spark32) { assert( plans(1) .asInstanceOf[HashJoinLikeExecTransformer] @@ -360,7 +360,7 @@ class GlutenClickHouseTPCHParquetBucketSuite .right .isInstanceOf[InputIteratorTransformer]) - if (sparkVersion.equals("3.2")) { + if (spark32) { assert(!plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } else { assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) @@ -381,7 +381,7 @@ class GlutenClickHouseTPCHParquetBucketSuite 3, tpchQueries, df => { - if (sparkVersion.equals("3.3")) { + if (spark33) { val plans = collectWithSubqueries(df.queryExecution.executedPlan) { case aggExec: HashAggregateExecBaseTransformer if aggExec.aggregateExpressions.exists( @@ -433,7 +433,7 @@ class GlutenClickHouseTPCHParquetBucketSuite 4, tpchQueries, df => { - if (sparkVersion.equals("3.3")) { + if (spark33) { val plans = collectWithSubqueries(df.queryExecution.executedPlan) { case aggExec: HashAggregateExecBaseTransformer if aggExec.aggregateExpressions.exists( @@ -501,7 +501,7 @@ class GlutenClickHouseTPCHParquetBucketSuite 12, tpchQueries, df => { - if (sparkVersion.equals("3.3")) { + if (spark33) { val plans = collectWithSubqueries(df.queryExecution.executedPlan) { case aggExec: HashAggregateExecBaseTransformer if aggExec.aggregateExpressions.exists( @@ -557,7 +557,7 @@ class GlutenClickHouseTPCHParquetBucketSuite val plans = collect(df.queryExecution.executedPlan) { case joinExec: HashJoinLikeExecTransformer => joinExec } - if (sparkVersion.equals("3.2")) { + if (spark32) { assert( plans(1) .asInstanceOf[HashJoinLikeExecTransformer] @@ -607,7 +607,7 @@ class GlutenClickHouseTPCHParquetBucketSuite 20, tpchQueries, df => { - if (sparkVersion.equals("3.3")) { + if (spark33) { val plans = collectWithSubqueries(df.queryExecution.executedPlan) { case aggExec: HashAggregateExecBaseTransformer if aggExec.aggregateExpressions.exists( diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetRFSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetRFSuite.scala index eb4118689fef..3dca8c11fd6c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetRFSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetRFSuite.scala @@ -56,7 +56,7 @@ class GlutenClickHouseTPCHParquetRFSuite extends GlutenClickHouseTPCHSaltNullPar |""".stripMargin, compareResult = true, df => { - if (sparkVersion.equals("3.3")) { + if (spark33) { val filterExecs = df.queryExecution.executedPlan.collect { case filter: FilterExecTransformerBase => filter } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 0536d9136ec8..679d2ddd85c4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -17,6 +17,7 @@ package org.apache.gluten.execution.tpch import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution._ import org.apache.gluten.extension.GlutenPlan @@ -41,9 +42,6 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr rootPath + "../../../../gluten-core/src/test/resources/tpch-queries" override protected val queriesResults: String = rootPath + "queries-output" - protected val BACKEND_CONF_KEY = "spark.gluten.sql.columnar.backend.ch." - protected val BACKEND_RUNTIME_CINF_KEY: String = BACKEND_CONF_KEY + "runtime_config." - override protected def sparkConf: SparkConf = { super.sparkConf .set("spark.shuffle.manager", "sort") @@ -1419,8 +1417,8 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr queriesResults: String = queriesResults, compareResult: Boolean = true, noFallBack: Boolean = true)(customCheck: DataFrame => Unit): Unit = { - val confName = "spark.gluten.sql.columnar.backend.ch." + - "runtime_settings.query_plan_enable_optimizations" + val confName = CHConf.runtimeSettings("query_plan_enable_optimizations") + withSQLConf((confName, "true")) { compareTPCHQueryAgainstVanillaSpark(queryNum, tpchQueries, customCheck, noFallBack) } @@ -2549,9 +2547,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr test("GLUTEN-4521: Invalid result from grace mergeing aggregation with spill") { withSQLConf( - ( - BACKEND_RUNTIME_CINF_KEY + "max_allowed_memory_usage_ratio_for_aggregate_merging", - "0.0001")) { + (CHConf.runtimeConfig("max_allowed_memory_usage_ratio_for_aggregate_merging"), "0.0001")) { val sql = """ |select count(l_orderkey, l_partkey) from ( @@ -2840,7 +2836,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark( sql, - true, + compareResult = true, df => { checkBHJWithIsNullAwareAntiJoin(df) }) @@ -2879,7 +2875,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark( sql1, - true, + compareResult = true, df => { checkBHJWithIsNullAwareAntiJoin(df) }) @@ -2892,7 +2888,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark( sql2, - true, + compareResult = true, df => { checkBHJWithIsNullAwareAntiJoin(df) }) @@ -2905,7 +2901,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark( sql3, - true, + compareResult = true, df => { checkBHJWithIsNullAwareAntiJoin(df) }) @@ -2918,7 +2914,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark( sql4, - true, + compareResult = true, df => { checkBHJWithIsNullAwareAntiJoin(df) }) @@ -2931,7 +2927,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark( sql5, - true, + compareResult = true, df => { checkBHJWithIsNullAwareAntiJoin(df) }) @@ -2952,7 +2948,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark( sql6, - true, + compareResult = true, df => { checkAQEBHJWithIsNullAwareAntiJoin(df, 0) }) @@ -2965,7 +2961,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark( sql7, - true, + compareResult = true, df => { checkAQEBHJWithIsNullAwareAntiJoin(df, 0) }) @@ -2978,7 +2974,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark( sql8, - true, + compareResult = true, df => { checkAQEBHJWithIsNullAwareAntiJoin(df) }) diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHOptimizeRuleBenchmark.scala b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHOptimizeRuleBenchmark.scala index 8d6d749fd650..a6bf4a9c994a 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHOptimizeRuleBenchmark.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHOptimizeRuleBenchmark.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.benchmarks +import org.apache.gluten.backendsapi.clickhouse.CHConf + import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark @@ -60,7 +62,7 @@ object CHOptimizeRuleBenchmark extends SqlBasedBenchmark with CHSqlBasedBenchmar } def testToDateOptimize(parquetDir: String, enable: String): Unit = { - withSQLConf(("spark.gluten.sql.columnar.backend.ch.rewrite.dateConversion", enable)) { + withSQLConf((CHConf.prefixOf("rewrite.dateConversion"), enable)) { spark .sql(s""" |select