From 65543fcfbcede89a4fb925bcceb682eb1b669edd Mon Sep 17 00:00:00 2001 From: yikaifei Date: Thu, 9 Jan 2025 10:04:45 +0800 Subject: [PATCH] Remove unnecessary config --- .../scala/org/apache/gluten/RunTPCHTest.scala | 1 - .../execution/GlutenClickHouseJoinSuite.scala | 1 - .../GlutenClickHouseSyntheticDataSuite.scala | 1 - .../GlutenClickHouseTPCDSAbstractSuite.scala | 1 - .../GlutenClickHouseTPCHAbstractSuite.scala | 1 - .../GlutenFunctionValidateSuite.scala | 1 - .../GlutenClickhouseFunctionSuite.scala | 1 - .../hive/GlutenClickHouseHiveTableSuite.scala | 1 - ...lutenClickHouseNativeWriteTableSuite.scala | 1 - .../benchmarks/CHSqlBasedBenchmark.scala | 1 - docs/get-started/ClickHouse.md | 4 - .../src/resources/conf/gluten.properties | 1 - .../benchmarks/ParquetReadBenchmark.scala | 1 - .../benchmarks/ParquetReadBenchmark.scala | 1 - .../benchmarks/ParquetReadBenchmark.scala | 1 - .../benchmarks/ParquetReadBenchmark.scala | 1 - .../apache/gluten/config/GlutenConfig.scala | 83 +++++++------------ .../native_sql_initialize.ipynb | 1 - 18 files changed, 32 insertions(+), 71 deletions(-) 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 6b8aee8b3520..7edd69e2b476 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/RunTPCHTest.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/RunTPCHTest.scala @@ -93,7 +93,6 @@ object RunTPCHTest { .config("spark.databricks.delta.snapshotPartitions", 1) .config("spark.databricks.delta.properties.defaults.checkpointInterval", 5) .config("spark.databricks.delta.stalenessLimit", 3600 * 1000) - .config("spark.gluten.sql.columnar.columnarToRow", columnarColumnToRow) .config(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .config(GlutenConfig.GLUTEN_LIB_PATH, libPath) .config("spark.gluten.sql.columnar.iterator", "true") 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 9a3e14270d87..b1ed60d210e6 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 @@ -42,7 +42,6 @@ class GlutenClickHouseJoinSuite extends GlutenClickHouseWholeStageTransformerSui .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.adaptive.enabled", "false") .set("spark.sql.files.minPartitionNum", "1") - .set("spark.gluten.sql.columnar.columnartorow", "true") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") 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 ac874bf88a01..ed72136c5124 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 @@ -51,7 +51,6 @@ class GlutenClickHouseSyntheticDataSuite .set("spark.databricks.delta.snapshotPartitions", "1") .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") - .set("spark.gluten.sql.columnar.columnarToRow", "true") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") 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 70ce5e6eb49a..7e377e6798fd 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 @@ -131,7 +131,6 @@ abstract class GlutenClickHouseTPCDSAbstractSuite .set("spark.databricks.delta.snapshotPartitions", "1") .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") - .set("spark.gluten.sql.columnar.columnarToRow", "true") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "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 08f743bb918a..63e801d20f3b 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 @@ -567,7 +567,6 @@ abstract class GlutenClickHouseTPCHAbstractSuite .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") - .set("spark.gluten.sql.columnar.columnarToRow", "true") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") 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 d900bc000c05..3eb9b6e36909 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 @@ -56,7 +56,6 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS .set("spark.databricks.delta.snapshotPartitions", "1") .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") - .set("spark.gluten.sql.columnar.columnartorow", "true") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") 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 6238c935b28d..07fd0b6ef828 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 @@ -50,7 +50,6 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { .set("spark.databricks.delta.snapshotPartitions", "1") .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") - .set("spark.gluten.sql.columnar.columnartorow", "true") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "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 21289f35d430..90b86a54703d 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 @@ -55,7 +55,6 @@ class GlutenClickHouseHiveTableSuite .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.adaptive.enabled", "false") .set("spark.sql.files.minPartitionNum", "1") - .set("spark.gluten.sql.columnar.columnartorow", "true") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") 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 4de1168ce31c..da0d82ef81fa 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 @@ -56,7 +56,6 @@ class GlutenClickHouseNativeWriteTableSuite .set("spark.databricks.delta.snapshotPartitions", "1") .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") - .set("spark.gluten.sql.columnar.columnartorow", "true") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") .set(GlutenConfig.GLUTEN_LIB_PATH, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHSqlBasedBenchmark.scala b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHSqlBasedBenchmark.scala index 297dffabbf74..6041c50d4126 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHSqlBasedBenchmark.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHSqlBasedBenchmark.scala @@ -43,7 +43,6 @@ trait CHSqlBasedBenchmark extends SqlBasedBenchmark { .set("spark.databricks.delta.snapshotPartitions", "1") .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.enable.native.validation", "false") .set("spark.sql.adaptive.enabled", "false") .setIfMissing("spark.memory.offHeap.size", offheapSize) diff --git a/docs/get-started/ClickHouse.md b/docs/get-started/ClickHouse.md index 287747418294..221a65cc395c 100644 --- a/docs/get-started/ClickHouse.md +++ b/docs/get-started/ClickHouse.md @@ -241,7 +241,6 @@ cd spark-3.2.2-bin-hadoop2.7 --conf spark.memory.offHeap.enabled=true \ --conf spark.memory.offHeap.size=6442450944 \ --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.gluten.sql.columnar.columnarToRow=true \ --conf spark.executorEnv.LD_PRELOAD=/path_to_clickhouse_library/libch.so\ --conf spark.gluten.sql.columnar.libpath=/path_to_clickhouse_library/libch.so \ --conf spark.gluten.sql.columnar.iterator=true \ @@ -422,7 +421,6 @@ cd spark-3.2.2-bin-hadoop2.7 --conf spark.memory.offHeap.enabled=true \ --conf spark.memory.offHeap.size=6442450944 \ --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.gluten.sql.columnar.columnarToRow=true \ --conf spark.executorEnv.LD_PRELOAD=/path_to_clickhouse_library/libch.so\ --conf spark.gluten.sql.columnar.libpath=/path_to_clickhouse_library/libch.so \ --conf spark.gluten.sql.columnar.iterator=true \ @@ -494,7 +492,6 @@ $spark_cmd \ --conf spark.sql.shuffle.partitions=112 \ --conf spark.sql.sources.useV1SourceList=avro \ --conf spark.sql.files.maxPartitionBytes=1073741824 \ - --conf spark.gluten.sql.columnar.columnartorow=true \ --conf spark.gluten.sql.columnar.loadnative=true \ --conf spark.gluten.sql.columnar.libpath=$ch_lib \ --conf spark.gluten.sql.columnar.iterator=true \ @@ -614,7 +611,6 @@ cd spark-3.2.2-bin-hadoop2.7 --conf spark.serializer=org.apache.spark.serializer.JavaSerializer \ --conf spark.sql.sources.ignoreDataLocality=true \ --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.gluten.sql.columnar.columnarToRow=true \ --conf spark.gluten.sql.columnar.libpath=/path_to_clickhouse_library/libch.so \ --conf spark.gluten.sql.columnar.iterator=true \ --conf spark.gluten.sql.columnar.loadarrow=false \ diff --git a/ep/build-clickhouse/src/resources/conf/gluten.properties b/ep/build-clickhouse/src/resources/conf/gluten.properties index ff613e821258..88650b17166d 100644 --- a/ep/build-clickhouse/src/resources/conf/gluten.properties +++ b/ep/build-clickhouse/src/resources/conf/gluten.properties @@ -11,7 +11,6 @@ spark.sql.columnVector.offheap.enabled=true spark.memory.offHeap.enabled=true spark.memory.offHeap.size=6442450944 spark.plugins=org.apache.gluten.GlutenPlugin -spark.gluten.sql.columnar.columnarToRow=true spark.gluten.sql.columnar.iterator=true spark.gluten.sql.columnar.loadarrow=false spark.gluten.sql.columnar.hashagg.enablefinal=true diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index de4dce405874..88e4d33f1227 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -70,7 +70,6 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { .set("spark.memory.offHeap.enabled", "true") .setIfMissing("spark.memory.offHeap.size", offheapSize) .setIfMissing("spark.sql.columnVector.offheap.enabled", "true") - .set("spark.gluten.sql.columnar.columnarToRow", "true") .set("spark.sql.adaptive.enabled", "false") .setIfMissing("spark.driver.memory", memorySize) .setIfMissing("spark.executor.memory", memorySize) diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index 85aa24a56392..68e86c1ba18f 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -70,7 +70,6 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { .set("spark.memory.offHeap.enabled", "true") .setIfMissing("spark.memory.offHeap.size", offheapSize) .setIfMissing("spark.sql.columnVector.offheap.enabled", "true") - .set("spark.gluten.sql.columnar.columnarToRow", "true") .set("spark.sql.adaptive.enabled", "false") .setIfMissing("spark.driver.memory", memorySize) .setIfMissing("spark.executor.memory", memorySize) diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index 8ddd85a0f232..f7c134a98e34 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -71,7 +71,6 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { .set("spark.memory.offHeap.enabled", "true") .setIfMissing("spark.memory.offHeap.size", offheapSize) .setIfMissing("spark.sql.columnVector.offheap.enabled", "true") - .set("spark.gluten.sql.columnar.columnarToRow", "true") .set("spark.sql.adaptive.enabled", "false") .setIfMissing("spark.driver.memory", memorySize) .setIfMissing("spark.executor.memory", memorySize) diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index 85aa24a56392..68e86c1ba18f 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -70,7 +70,6 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { .set("spark.memory.offHeap.enabled", "true") .setIfMissing("spark.memory.offHeap.size", offheapSize) .setIfMissing("spark.sql.columnVector.offheap.enabled", "true") - .set("spark.gluten.sql.columnar.columnarToRow", "true") .set("spark.sql.adaptive.enabled", "false") .setIfMissing("spark.driver.memory", memorySize) .setIfMissing("spark.executor.memory", memorySize) diff --git a/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala index cee4ea7d1645..f31745a67c9c 100644 --- a/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala @@ -85,8 +85,6 @@ class GlutenConfig(conf: SQLConf) extends Logging { def shuffledHashJoinOptimizeBuildSide: Boolean = getConf(COLUMNAR_SHUFFLED_HASH_JOIN_OPTIMIZE_BUILD_SIDE) - def enableNativeColumnarToRow: Boolean = getConf(COLUMNAR_COLUMNAR_TO_ROW_ENABLED) - def forceShuffledHashJoin: Boolean = getConf(COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED) def enableColumnarSortMergeJoin: Boolean = getConf(COLUMNAR_SORTMERGEJOIN_ENABLED) @@ -169,17 +167,12 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enablePreferColumnar: Boolean = getConf(COLUMNAR_PREFER_ENABLED) - def enableOneRowRelationColumnar: Boolean = getConf(COLUMNAR_ONE_ROW_RELATION_ENABLED) - def physicalJoinOptimizationThrottle: Integer = getConf(COLUMNAR_PHYSICAL_JOIN_OPTIMIZATION_THROTTLE) def enablePhysicalJoinOptimize: Boolean = getConf(COLUMNAR_PHYSICAL_JOIN_OPTIMIZATION_ENABLED) - def logicalJoinOptimizationThrottle: Integer = - getConf(COLUMNAR_LOGICAL_JOIN_OPTIMIZATION_THROTTLE) - def enableScanOnly: Boolean = getConf(COLUMNAR_SCAN_ONLY_ENABLED) def tmpFile: Option[String] = getConf(COLUMNAR_TEMP_DIR) @@ -371,7 +364,7 @@ class GlutenConfig(conf: SQLConf) extends Logging { def chColumnarShuffleSpillThreshold: Long = { val threshold = getConf(COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD) if (threshold == 0) { - (getConf(COLUMNAR_TASK_OFFHEAP_SIZE_IN_BYTES) * 0.9).toLong + (taskOffHeapMemorySize * 0.9).toLong } else { threshold } @@ -432,47 +425,56 @@ class GlutenConfig(conf: SQLConf) extends Logging { getConf(COLUMNAR_VELOX_MEMORY_USE_HUGE_PAGES) def debug: Boolean = getConf(DEBUG_ENABLED) + def debugKeepJniWorkspace: Boolean = getConf(DEBUG_KEEP_JNI_WORKSPACE) + def collectUtStats: Boolean = getConf(UT_STATISTIC) + def benchmarkStageId: Int = getConf(BENCHMARK_TASK_STAGEID) + def benchmarkPartitionId: String = getConf(BENCHMARK_TASK_PARTITIONID) + def benchmarkTaskId: String = getConf(BENCHMARK_TASK_TASK_ID) + def benchmarkSaveDir: String = getConf(BENCHMARK_SAVE_DIR) + def textInputMaxBlockSize: Long = getConf(TEXT_INPUT_ROW_MAX_BLOCK_SIZE) + def textIputEmptyAsDefault: Boolean = getConf(TEXT_INPUT_EMPTY_AS_DEFAULT) + def enableParquetRowGroupMaxMinIndex: Boolean = getConf(ENABLE_PARQUET_ROW_GROUP_MAX_MIN_INDEX) def enableVeloxFlushablePartialAggregation: Boolean = getConf(VELOX_FLUSHABLE_PARTIAL_AGGREGATION_ENABLED) - def maxFlushableAggregationMemoryRatio: Double = - getConf(MAX_PARTIAL_AGGREGATION_MEMORY_RATIO) - def maxExtendedFlushableAggregationMemoryRatio: Double = - getConf(MAX_PARTIAL_AGGREGATION_MEMORY_RATIO) - def abandonFlushableAggregationMinPct: Int = - getConf(ABANDON_PARTIAL_AGGREGATION_MIN_PCT) - def abandonFlushableAggregationMinRows: Int = - getConf(ABANDON_PARTIAL_AGGREGATION_MIN_ROWS) + + def maxFlushableAggregationMemoryRatio: Double = getConf(MAX_PARTIAL_AGGREGATION_MEMORY_RATIO) + + def maxExtendedFlushableAggregationMemoryRatio: Double = getConf( + MAX_PARTIAL_AGGREGATION_MEMORY_RATIO) + + def abandonFlushableAggregationMinPct: Int = getConf(ABANDON_PARTIAL_AGGREGATION_MIN_PCT) + + def abandonFlushableAggregationMinRows: Int = getConf(ABANDON_PARTIAL_AGGREGATION_MIN_ROWS) // Please use `BackendsApiManager.getSettings.enableNativeWriteFiles()` instead def enableNativeWriter: Option[Boolean] = getConf(NATIVE_WRITER_ENABLED) def enableNativeArrowReader: Boolean = getConf(NATIVE_ARROW_READER_ENABLED) - def directorySizeGuess: Long = - getConf(DIRECTORY_SIZE_GUESS) - def filePreloadThreshold: Long = - getConf(FILE_PRELOAD_THRESHOLD) - def prefetchRowGroups: Int = - getConf(PREFETCH_ROW_GROUPS) - def loadQuantum: Long = - getConf(LOAD_QUANTUM) - def maxCoalescedDistance: String = - getConf(MAX_COALESCED_DISTANCE_BYTES) - def maxCoalescedBytes: Long = - getConf(MAX_COALESCED_BYTES) - def cachePrefetchMinPct: Int = - getConf(CACHE_PREFETCH_MINPCT) + def directorySizeGuess: Long = getConf(DIRECTORY_SIZE_GUESS) + + def filePreloadThreshold: Long = getConf(FILE_PRELOAD_THRESHOLD) + + def prefetchRowGroups: Int = getConf(PREFETCH_ROW_GROUPS) + + def loadQuantum: Long = getConf(LOAD_QUANTUM) + + def maxCoalescedDistance: String = getConf(MAX_COALESCED_DISTANCE_BYTES) + + def maxCoalescedBytes: Long = getConf(MAX_COALESCED_BYTES) + + def cachePrefetchMinPct: Int = getConf(CACHE_PREFETCH_MINPCT) def enableColumnarProjectCollapse: Boolean = getConf(ENABLE_COLUMNAR_PROJECT_COLLAPSE) @@ -1032,13 +1034,6 @@ object GlutenConfig { .booleanConf .createWithDefault(true) - val COLUMNAR_COLUMNAR_TO_ROW_ENABLED = - buildConf("spark.gluten.sql.columnar.columnarToRow") - .internal() - .doc("Enable or disable columnar columnarToRow.") - .booleanConf - .createWithDefault(true) - val COLUMNAR_SORTMERGEJOIN_ENABLED = buildConf("spark.gluten.sql.columnar.sortMergeJoin") .internal() @@ -1126,13 +1121,6 @@ object GlutenConfig { .booleanConf .createWithDefault(true) - val COLUMNAR_ONE_ROW_RELATION_ENABLED = - buildConf("spark.gluten.sql.columnar.oneRowRelation") - .internal() - .doc("Enable or disable columnar `OneRowRelation`.") - .booleanConf - .createWithDefault(true) - val COLUMNAR_TABLE_CACHE_ENABLED = buildConf("spark.gluten.sql.columnar.tableCache") .internal() @@ -1154,13 +1142,6 @@ object GlutenConfig { .booleanConf .createWithDefault(false) - val COLUMNAR_LOGICAL_JOIN_OPTIMIZATION_THROTTLE = - buildConf("spark.gluten.sql.columnar.logicalJoinOptimizationLevel") - .internal() - .doc("Fallback to row operators if there are several continuous joins.") - .intConf - .createWithDefault(12) - val COLUMNAR_SCAN_ONLY_ENABLED = buildConf("spark.gluten.sql.columnar.scanOnly") .internal() diff --git a/tools/workload/benchmark_velox/native_sql_initialize.ipynb b/tools/workload/benchmark_velox/native_sql_initialize.ipynb index f606260c95f6..6f4f54649e5d 100644 --- a/tools/workload/benchmark_velox/native_sql_initialize.ipynb +++ b/tools/workload/benchmark_velox/native_sql_initialize.ipynb @@ -1276,7 +1276,6 @@ " .set('spark.gluten.sql.columnar.physicalJoinOptimizeEnable', 'true')\\\n", " .set('spark.gluten.sql.columnar.physicalJoinOptimizationLevel', '18')\\\n", " .set('spark.gluten.sql.columnar.logicalJoinOptimizeEnable', 'true')\\\n", - " .set('spark.gluten.sql.columnar.logicalJoinOptimizationLevel', '19')\n", " return conf" ] },