layout: page title: Configuration nav_order: 15 ————-

Spark base configurations for Gluten plugin

Key Recommend Setting Description
spark.plugins org.apache.gluten.GlutenPlugin To load Gluten’s components by Spark’s plug-in loader.
spark.memory.offHeap.enabled true Gluten use off-heap memory for certain operations.
spark.memory.offHeap.size 30G The absolute amount of memory which can be used for off-heap allocation, in bytes unless otherwise specified.
Note: Gluten Plugin will leverage this setting to allocate memory space for native usage even offHeap is disabled.
The value is based on your system and it is recommended to set it larger if you are facing Out of Memory issue in Gluten Plugin.
spark.shuffle.manager org.apache.spark.shuffle.sort.ColumnarShuffleManager To turn on Gluten Columnar Shuffle Plugin.
spark.driver.extraClassPath /path/to/gluten_jar_file Gluten Plugin jar file to prepend to the classpath of the driver.
spark.executor.extraClassPath /path/to/gluten_jar_file Gluten Plugin jar file to prepend to the classpath of executors.

Gluten configurations

Key Default Description
spark.gluten.enabled true Whether to enable gluten. Default value is true. Just an experimental property. Recommend to enable/disable Gluten through the setting for spark.plugins.
spark.gluten.execution.resource.expired.time 86400 Expired time of execution with resource relation has cached.
spark.gluten.expression.blacklist <undefined> A black list of expression to skip transform, multiple values separated by commas.
spark.gluten.loadLibFromJar false Whether to load shared libraries from jars.
spark.gluten.loadLibOS <undefined> The shared library loader’s OS name.
spark.gluten.loadLibOSVersion <undefined> The shared library loader’s OS version.
spark.gluten.memory.isolation false Enable isolated memory mode. If true, Gluten controls the maximum off-heap memory can be used by each task to X, X = executor memory / max task slots. It’s recommended to set true if Gluten serves concurrent queries within a single session, since not all memory Gluten allocated is guaranteed to be spillable. In the case, the feature should be enabled to avoid OOM.
spark.gluten.memory.overAcquiredMemoryRatio 0.3 If larger than 0, Velox backend will try over-acquire this ratio of the total allocated memory as backup to avoid OOM.
spark.gluten.memory.reservationBlockSize 8MB Block size of native reservation listener reserve memory from Spark.
spark.gluten.numTaskSlotsPerExecutor -1 Must provide default value since non-execution operations (e.g. org.apache.spark.sql.Dataset#summary) doesn’t propagate configurations using org.apache.spark.sql.execution.SQLExecution#withSQLConfPropagated
spark.gluten.ras.costModel legacy The class name of user-defined cost model that will be used by Gluten’s transition planner as well as by RAS. If not specified, a legacy built-in cost model will be used. The legacy cost model helps RAS planner exhaustively offload computations, and helps transition planner choose columnar-to-columnar transition over others.
spark.gluten.ras.enabled false Enables RAS (relational algebra selector) during physical planning to generate more efficient query plan. Note, this feature doesn’t bring performance profits by default. Try exploring option spark.gluten.ras.costModel for advanced usage.
spark.gluten.shuffleWriter.bufferSize <undefined>  
spark.gluten.soft-affinity.duplicateReading.maxCacheItems 10000 Enable Soft Affinity duplicate reading detection
spark.gluten.soft-affinity.duplicateReadingDetect.enabled false If true, Enable Soft Affinity duplicate reading detection
spark.gluten.soft-affinity.enabled false Whether to enable Soft Affinity scheduling.
spark.gluten.soft-affinity.min.target-hosts 1 For on HDFS, if there are already target hosts, and then prefer to use the original target hosts to schedule
spark.gluten.soft-affinity.replications.num 2 Calculate the number of the replications for scheduling to the target executors per file
spark.gluten.sql.adaptive.costEvaluator.enabled true If true, use org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator as custom cost evaluator class, else follow the configuration spark.sql.adaptive.customCostEvaluatorClass.
spark.gluten.sql.ansiFallback.enabled true When true (default), Gluten will fall back to Spark when ANSI mode is enabled. When false, Gluten will attempt to execute in ANSI mode.
spark.gluten.sql.broadcastNestedLoopJoinTransformerEnabled true Config to enable BroadcastNestedLoopJoinExecTransformer.
spark.gluten.sql.cacheWholeStageTransformerContext false When true, WholeStageTransformer will cache the WholeStageTransformerContext when executing. It is used to get substrait plan node and native plan string.
spark.gluten.sql.cartesianProductTransformerEnabled true Config to enable CartesianProductExecTransformer.
spark.gluten.sql.collapseGetJsonObject.enabled false Collapse nested get_json_object functions as one for optimization.
spark.gluten.sql.columnar.appendData true Enable or disable columnar v2 command append data.
spark.gluten.sql.columnar.arrowUdf true Enable or disable columnar arrow udf.
spark.gluten.sql.columnar.batchscan true Enable or disable columnar batchscan.
spark.gluten.sql.columnar.broadcastExchange true Enable or disable columnar broadcastExchange.
spark.gluten.sql.columnar.broadcastJoin true Enable or disable columnar broadcastJoin.
spark.gluten.sql.columnar.cast.avg true  
spark.gluten.sql.columnar.coalesce true Enable or disable columnar coalesce.
spark.gluten.sql.columnar.collectLimit true Enable or disable columnar collectLimit.
spark.gluten.sql.columnar.collectTail true Enable or disable columnar collectTail.
spark.gluten.sql.columnar.enableNestedColumnPruningInHiveTableScan true Enable or disable nested column pruning in hivetablescan.
spark.gluten.sql.columnar.enableVanillaVectorizedReaders true Enable or disable vanilla vectorized scan.
spark.gluten.sql.columnar.executor.libpath   The gluten executor library path.
spark.gluten.sql.columnar.expand true Enable or disable columnar expand.
spark.gluten.sql.columnar.fallback.expressions.threshold 50 Fall back filter/project if number of nested expressions reaches this threshold, considering Spark codegen can bring better performance for such case.
spark.gluten.sql.columnar.fallback.ignoreRowToColumnar true When true, the fallback policy ignores the RowToColumnar when counting fallback number.
spark.gluten.sql.columnar.fallback.preferColumnar true When true, the fallback policy prefers to use Gluten plan rather than vanilla Spark plan if the both of them contains ColumnarToRow and the vanilla Spark plan ColumnarToRow number is not smaller than Gluten plan.
spark.gluten.sql.columnar.filescan true Enable or disable columnar filescan.
spark.gluten.sql.columnar.filter true Enable or disable columnar filter.
spark.gluten.sql.columnar.force.hashagg true Whether to force to use gluten’s hash agg for replacing vanilla spark’s sort agg.
spark.gluten.sql.columnar.forceShuffledHashJoin true  
spark.gluten.sql.columnar.generate true  
spark.gluten.sql.columnar.hashagg true Enable or disable columnar hashagg.
spark.gluten.sql.columnar.hivetablescan true Enable or disable columnar hivetablescan.
spark.gluten.sql.columnar.libname gluten The gluten library name.
spark.gluten.sql.columnar.libpath   The gluten library path.
spark.gluten.sql.columnar.limit true  
spark.gluten.sql.columnar.maxBatchSize 4096  
spark.gluten.sql.columnar.overwriteByExpression true Enable or disable columnar v2 command overwrite by expression.
spark.gluten.sql.columnar.overwritePartitionsDynamic true Enable or disable columnar v2 command overwrite partitions dynamic.
spark.gluten.sql.columnar.parquet.write.blockSize 128MB  
spark.gluten.sql.columnar.partial.generate true Evaluates the non-offload-able HiveUDTF using vanilla Spark generator
spark.gluten.sql.columnar.partial.project true Break up one project node into 2 phases when some of the expressions are non offload-able. Phase one is a regular offloaded project transformer that evaluates the offload-able expressions in native, phase two preserves the output from phase one and evaluates the remaining non-offload-able expressions using vanilla Spark projections
spark.gluten.sql.columnar.physicalJoinOptimizationLevel 12 Fallback to row operators if there are several continuous joins.
spark.gluten.sql.columnar.physicalJoinOptimizeEnable false Enable or disable columnar physicalJoinOptimize.
spark.gluten.sql.columnar.preferStreamingAggregate true Velox backend supports StreamingAggregate. StreamingAggregate uses the less memory as it does not need to hold all groups in memory, so it could avoid spill. When true and the child output ordering satisfies the grouping key then Gluten will choose StreamingAggregate as the native operator.
spark.gluten.sql.columnar.project true Enable or disable columnar project.
spark.gluten.sql.columnar.project.collapse true Combines two columnar project operators into one and perform alias substitution
spark.gluten.sql.columnar.query.fallback.threshold -1 The threshold for whether query will fall back by counting the number of ColumnarToRow & vanilla leaf node.
spark.gluten.sql.columnar.range true Enable or disable columnar range.
spark.gluten.sql.columnar.replaceData true Enable or disable columnar v2 command replace data.
spark.gluten.sql.columnar.scanOnly false When enabled, only scan and the filter after scan will be offloaded to native.
spark.gluten.sql.columnar.shuffle true Enable or disable columnar shuffle.
spark.gluten.sql.columnar.shuffle.celeborn.fallback.enabled true If enabled, fall back to ColumnarShuffleManager when celeborn service is unavailable.Otherwise, throw an exception.
spark.gluten.sql.columnar.shuffle.celeborn.useRssSort true If true, use RSS sort implementation for Celeborn sort-based shuffle.If false, use Gluten’s row-based sort implementation. Only valid when spark.celeborn.client.spark.shuffle.writer is set to sort.
spark.gluten.sql.columnar.shuffle.codec <undefined> By default, the supported codecs are lz4 and zstd. When spark.gluten.sql.columnar.shuffle.codecBackend=qat,the supported codecs are gzip and zstd.
spark.gluten.sql.columnar.shuffle.codecBackend <undefined>  
spark.gluten.sql.columnar.shuffle.compression.threshold 100 If number of rows in a batch falls below this threshold, will copy all buffers into one buffer to compress.
spark.gluten.sql.columnar.shuffle.dictionary.enabled false Enable dictionary in hash-based shuffle.
spark.gluten.sql.columnar.shuffle.merge.threshold 0.25  
spark.gluten.sql.columnar.shuffle.readerBufferSize 1MB Buffer size in bytes for shuffle reader reading input stream from local or remote.
spark.gluten.sql.columnar.shuffle.realloc.threshold 0.25  
spark.gluten.sql.columnar.shuffle.sort.columns.threshold 100000 The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of columns is greater than this threshold.
spark.gluten.sql.columnar.shuffle.sort.deserializerBufferSize 1MB Buffer size in bytes for sort-based shuffle reader deserializing raw input to columnar batch.
spark.gluten.sql.columnar.shuffle.sort.partitions.threshold 4000 The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of partitions is greater than this threshold.
spark.gluten.sql.columnar.shuffledHashJoin true Enable or disable columnar shuffledHashJoin.
spark.gluten.sql.columnar.shuffledHashJoin.optimizeBuildSide true Whether to allow Gluten to choose an optimal build side for shuffled hash join.
spark.gluten.sql.columnar.smallFileThreshold 0.5 The total size threshold of small files in table scan.To avoid small files being placed into the same partition, Gluten will try to distribute small files into different partitions when the total size of small files is below this threshold.
spark.gluten.sql.columnar.sort true Enable or disable columnar sort.
spark.gluten.sql.columnar.sortMergeJoin true Enable or disable columnar sortMergeJoin. This should be set with preferSortMergeJoin=false.
spark.gluten.sql.columnar.tableCache false Enable or disable columnar table cache.
spark.gluten.sql.columnar.takeOrderedAndProject true  
spark.gluten.sql.columnar.union true Enable or disable columnar union.
spark.gluten.sql.columnar.wholeStage.fallback.threshold -1 The threshold for whether whole stage will fall back in AQE supported case by counting the number of ColumnarToRow & vanilla leaf node.
spark.gluten.sql.columnar.window true Enable or disable columnar window.
spark.gluten.sql.columnar.window.group.limit true Enable or disable columnar window group limit.
spark.gluten.sql.columnarSampleEnabled false Disable or enable columnar sample.
spark.gluten.sql.columnarToRowMemoryThreshold 64MB  
spark.gluten.sql.countDistinctWithoutExpand false Convert Count Distinct to a UDAF called count_distinct to prevent SparkPlanner converting it to Expand+Count. WARNING: When enabled, count distinct queries will fail to fallback!!!
spark.gluten.sql.extendedColumnPruning.enabled true Do extended nested column pruning for cases ignored by vanilla Spark.
spark.gluten.sql.fallbackRegexpExpressions false If true, fall back all regexp expressions. There are a few incompatible cases between RE2 (used by native engine) and java.util.regex (used by Spark). User should enable this property if their incompatibility is intolerable.
spark.gluten.sql.fallbackUnexpectedMetadataParquet false If enabled, Gluten will not offload scan when unexpected metadata is detected.
spark.gluten.sql.fallbackUnexpectedMetadataParquet.limit 10 If supplied, metadata of limit number of Parquet files will be checked to determine whether to fall back to java scan.
spark.gluten.sql.injectNativePlanStringToExplain false When true, Gluten will inject native plan tree to Spark’s explain output.
spark.gluten.sql.mergeTwoPhasesAggregate.enabled true Whether to merge two phases aggregate if there are no other operators between them.
spark.gluten.sql.native.arrow.reader.enabled false This is config to specify whether to enable the native columnar csv reader
spark.gluten.sql.native.bloomFilter true  
spark.gluten.sql.native.hive.writer.enabled true This is config to specify whether to enable the native columnar writer for HiveFileFormat. Currently only supports HiveFileFormat with Parquet as the output file type.
spark.gluten.sql.native.hyperLogLog.Aggregate true  
spark.gluten.sql.native.parquet.write.blockRows 100000000  
spark.gluten.sql.native.union false Enable or disable native union where computation is completely offloaded to backend.
spark.gluten.sql.native.writeColumnMetadataExclusionList comment Native write files does not support column metadata. Metadata in list would be removed to support native write files. Multiple values separated by commas.
spark.gluten.sql.native.writer.enabled <undefined> This is config to specify whether to enable the native columnar parquet/orc writer
spark.gluten.sql.orc.charType.scan.fallback.enabled true Force fallback for orc char type scan.
spark.gluten.sql.removeNativeWriteFilesSortAndProject true When true, Gluten will remove the vanilla Spark V1Writes added sort and project for velox backend.
spark.gluten.sql.rewrite.dateTimestampComparison true Rewrite the comparision between date and timestamp to timestamp comparison.For example from_unixtime(ts) > date will be rewritten to ts > to_unixtime(date)
spark.gluten.sql.scan.fileSchemeValidation.enabled true When true, enable file path scheme validation for scan. Validation will fail if file scheme is not supported by registered file systems, which will cause scan operator fall back.
spark.gluten.sql.supported.flattenNestedFunctions and,or Flatten nested functions as one for optimization.
spark.gluten.sql.text.input.empty.as.default false treat empty fields in CSV input as default values.
spark.gluten.sql.text.input.max.block.size 8KB the max block size for text input rows
spark.gluten.sql.validation.printStackOnFailure false  
spark.gluten.storage.hdfsViewfs.enabled false If enabled, gluten will convert the viewfs path to hdfs path in scala side
spark.gluten.supported.hive.udfs   Supported hive udf names.
spark.gluten.supported.python.udfs   Supported python udf names.
spark.gluten.supported.scala.udfs   Supported scala udf names.
spark.gluten.ui.enabled true Whether to enable the gluten web UI, If true, attach the gluten UI page to the Spark web UI.

Gluten experimental configurations

Key Default Description
spark.gluten.auto.adjustStageResource.enabled false Experimental: If enabled, gluten will try to set the stage resource according to stage execution plan. Only worked when aqe is enabled at the same time!!
spark.gluten.auto.adjustStageResources.fallenNode.ratio.threshold 0.5 Experimental: Increase executor heap memory when stage contains fallen node count exceeds the total node count ratio.
spark.gluten.auto.adjustStageResources.heap.ratio 2.0 Experimental: Increase executor heap memory when match adjust stage resource rule.
spark.gluten.auto.adjustStageResources.offheap.ratio 0.5 Experimental: Decrease executor offheap memory when match adjust stage resource rule.
spark.gluten.memory.dynamic.offHeap.sizing.enabled false Experimental: When set to true, the offheap config (spark.memory.offHeap.size) will be ignored and instead we will consider onheap and offheap memory in combination, both counting towards the executor memory config (spark.executor.memory). We will make use of JVM APIs to determine how much onheap memory is use, alongside tracking offheap allocations made by Gluten. We will then proceed to enforcing a total memory quota, calculated by the sum of what memory is committed and in use in the Java heap. Since the calculation of the total quota happens as offheap allocation happens and not as JVM heap memory is allocated, it is possible that we can oversubscribe memory. Additionally, note that this change is experimental and may have performance implications.
spark.gluten.memory.dynamic.offHeap.sizing.memory.fraction 0.6 Experimental: Determines the memory fraction used to determine the total memory available for offheap and onheap allocations when the dynamic offheap sizing feature is enabled. The default is set to match spark.executor.memoryFraction.
spark.gluten.sql.columnar.cudf false Enable or disable cudf support. This is an experimental feature.

This site uses Just the Docs, a documentation theme for Jekyll.