diff --git a/core/src/main/resources/bootstrap/tuningTable.yaml b/core/src/main/resources/bootstrap/tuningTable.yaml index 919a9a427..a40c79611 100644 --- a/core/src/main/resources/bootstrap/tuningTable.yaml +++ b/core/src/main/resources/bootstrap/tuningTable.yaml @@ -13,33 +13,38 @@ # limitations under the License. tuningDefinitions: + - label: spark.databricks.adaptive.autoOptimizeShuffle.enabled + description: 'Auto-Optimized shuffle. It is recommended to turn it off to set (spark.sql.shuffle.partitions) manually.' + enabled: true + level: job + category: tuning - label: spark.executor.cores - description: '' + description: 'The number of cores to use on each executor. It is recommended to be set to 16' enabled: true level: cluster category: tuning - label: spark.executor.instances - description: '' + description: 'Controls parellelism level. It is recommended to be set to (cpuCoresPerNode * numWorkers) / spark.executor.cores.' enabled: true level: cluster category: tuning - label: spark.executor.memory - description: '' + description: 'Amount of memory to use per executor process. This is tuned based on the available CPU memory on worker node.' enabled: true level: cluster category: tuning - label: spark.executor.memoryOverhead - description: '' + description: 'Amount of additional memory to be allocated per executor process, in MiB unless otherwise specified. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size.' enabled: true level: cluster category: tuning - label: spark.executor.memoryOverheadFactor - description: '' + description: 'Fraction of executor memory to be allocated as additional non-heap memory per executor process. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size.' enabled: true level: cluster category: tuning - label: spark.kubernetes.memoryOverheadFactor - description: '' + description: 'Specific to K8s. Fraction of executor memory to be allocated as additional non-heap memory per executor process.' enabled: true level: cluster category: tuning @@ -48,22 +53,24 @@ tuningDefinitions: enabled: true level: cluster category: tuning - defaultSpark: 3s + defaultSpark: "3s" - label: spark.rapids.filecache.enabled - description: '' + description: 'Enables RAPIDS file cache. The file cache stores data locally in the same local directories that have been configured for the Spark executor.' enabled: true level: job + category: tuning - label: spark.rapids.memory.pinnedPool.size - description: '' + description: 'The size of the pinned memory pool in bytes unless otherwise specified. Use 0 to disable the pool.' enabled: true level: cluster + category: tuning - label: spark.rapids.shuffle.multiThreaded.maxBytesInFlight - description: '' + description: 'This property controls the amount of bytes we allow in flight per Spark task. This typically happens on the reader side, when blocks are received from the network, they’re queued onto these threads for decompression and decode. ' enabled: true level: cluster category: tuning - label: spark.rapids.shuffle.multiThreaded.reader.threads - description: '' + description: 'The shuffle reader is a single implementation irrespective of the number of partitions. Set the value to zero to turn off multi-threaded reader entirely.' enabled: true level: cluster category: tuning @@ -73,86 +80,91 @@ tuningDefinitions: level: cluster category: tuning - label: spark.rapids.sql.batchSizeBytes - description: '' + description: 'Set the target number of bytes for a GPU batch. Splits sizes for input data is covered by separate configs.' enabled: true level: job category: tuning - label: spark.rapids.sql.concurrentGpuTasks - description: '' + description: 'Set the number of tasks that can execute concurrently per GPU. Tasks may temporarily block when the number of concurrent tasks in the executor exceeds this amount. Allowing too many concurrent tasks on the same GPU may lead to GPU out of memory errors.' enabled: true level: cluster category: tuning - label: spark.rapids.sql.format.parquet.multithreaded.combine.waitTime - description: '' + description: 'When using the multithreaded parquet reader with combine mode, how long to wait, in milliseconds, for more files to finish if haven’t met the size threshold. Note that this will wait this amount of time from when the last file was available, so total wait time could be larger then this. DEPRECATED: use spark.rapids.sql.reader.multithreaded.combine.waitTime instead.' enabled: true level: cluster category: tuning - label: spark.rapids.sql.enabled - description: 'should be true to enable SQL operations on the GPU.' + description: 'Should be true to enable SQL operations on the GPU.' enabled: true level: cluster category: functionality - label: spark.rapids.sql.multiThreadedRead.numThreads - description: '' + description: 'The maximum number of threads on each executor to use for reading small files in parallel.' enabled: true level: cluster category: tuning - label: spark.rapids.sql.reader.multithreaded.combine.sizeBytes - description: '' + description: 'The target size in bytes to combine multiple small files together when using the MULTITHREADED parquet or orc reader. With combine disabled, the MULTITHREADED reader reads the files in parallel and sends individual files down to the GPU, but that can be inefficient for small files.' enabled: true level: cluster category: tuning - label: spark.shuffle.manager - description: '' + description: 'The RAPIDS Shuffle Manager is an implementation of the ShuffleManager interface in Apache Spark that allows custom mechanisms to exchange shuffle data. We currently expose two modes of operation: Multi Threaded and UCX.' enabled: true level: cluster + category: tuning - label: spark.sql.adaptive.enabled - description: '' + description: 'When true, enable adaptive query execution, which re-optimizes the query plan in the middle of query execution, based on accurate runtime statistics.' enabled: true level: job category: tuning + defaultSpark: "true" - label: spark.sql.adaptive.advisoryPartitionSizeInBytes - description: '' + description: 'The advisory size in bytes of the shuffle partition during adaptive optimization (when spark.sql.adaptive.enabled is true). It takes effect when Spark coalesces small shuffle partitions or splits skewed shuffle partition.' enabled: true level: job category: tuning - label: spark.sql.adaptive.coalescePartitions.initialPartitionNum - description: '' + description: 'The initial number of shuffle partitions before coalescing. If not set, it equals to spark.sql.shuffle.partitions.' enabled: true level: job category: tuning - label: spark.sql.adaptive.coalescePartitions.minPartitionNum - description: '' + description: '(deprecated) The suggested (not guaranteed) minimum number of shuffle partitions after coalescing. If not set, the default value is the default parallelism of the Spark cluster.' enabled: true level: job category: tuning - label: spark.sql.adaptive.coalescePartitions.minPartitionSize - description: '' + description: 'The minimum size of shuffle partitions after coalescing. This is useful when the adaptively calculated target size is too small during partition coalescing.' enabled: true level: job category: tuning + defaultSpark: "1m" - label: spark.sql.adaptive.coalescePartitions.parallelismFirst - description: '' + description: 'When true, Spark does not respect the target size specified by (spark.sql.adaptive.advisoryPartitionSizeInBytes) (default 64MB) when coalescing contiguous shuffle partitions, but adaptively calculate the target size according to the default parallelism of the Spark cluster.' enabled: true level: job category: tuning + defaultSpark: "true" - label: spark.sql.adaptive.autoBroadcastJoinThreshold - description: '' + description: 'Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when performing a join. By setting this value to -1, broadcasting can be disabled.' enabled: true level: job category: tuning - label: spark.sql.files.maxPartitionBytes - description: '' + description: 'The maximum number of bytes to pack into a single partition when reading files. This configuration is effective only when using file-based sources such as Parquet, JSON and ORC.' enabled: true level: job category: tuning - label: spark.sql.shuffle.partitions - description: '' + description: 'The default number of partitions to use when shuffling data for joins or aggregations. Note: For structured streaming, this configuration cannot be changed between query restarts from the same checkpoint location.' enabled: true level: job category: tuning + defaultSpark: "200" - label: spark.task.resource.gpu.amount - description: '' + description: 'The GPU resource amount per task when Apache Spark schedules GPU resources. For example, setting the value to 1 means that only one task will run concurrently per executor.' enabled: true level: cluster category: tuning diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala index 74c9f1b5f..7047060c7 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala @@ -719,6 +719,7 @@ class AutoTuner( } private def recommendAQEProperties(): Unit = { + // Spark configuration (AQE is enabled by default) val aqeEnabled = getPropertyValue("spark.sql.adaptive.enabled") .getOrElse("false").toLowerCase if (aqeEnabled == "false") { diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala index 68dc72064..f1cd67f9c 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala @@ -95,7 +95,6 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 |--conf spark.task.resource.gpu.amount=0.001 | |Comments: @@ -115,10 +114,8 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- Number of workers is missing. Setting default to 1. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -141,7 +138,6 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.executor.cores' should be set to 16. @@ -154,7 +150,6 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' should be set to 0.001. |- Could not infer the cluster configuration, recommendations are generated using default values! |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -178,7 +173,6 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.executor.cores' should be set to 16. @@ -191,7 +185,6 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' should be set to 0.001. |- Could not infer the cluster configuration, recommendations are generated using default values! |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -234,7 +227,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.executor.cores' should be set to 16. @@ -247,7 +239,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' should be set to 0.001. |- Could not infer the cluster configuration, recommendations are generated using default values! |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -290,7 +281,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 |--conf spark.task.resource.gpu.amount=0.001 | |Comments: @@ -310,10 +300,8 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -355,7 +343,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 |--conf spark.task.resource.gpu.amount=0.001 | |Comments: @@ -375,10 +362,8 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- Number of workers is missing. Setting default to 1. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -427,7 +412,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -440,8 +424,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- GPU count is missing. Setting default to 1. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -492,7 +474,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -501,8 +482,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- GPU memory is missing. Setting default to 15109m. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -552,7 +531,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -560,7 +538,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- GPU memory is missing. Setting default to 15109m. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -609,7 +586,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -618,8 +594,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- GPU device is missing. Setting default to $T4Gpu. |- GPU memory is missing. Setting default to ${T4Gpu.getMemory}. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -670,7 +644,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -679,8 +652,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- GPU device is missing. Setting default to $T4Gpu. |- GPU memory is missing. Setting default to ${T4Gpu.getMemory}. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -725,7 +696,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 |--conf spark.task.resource.gpu.amount=0.001 | |Comments: @@ -745,10 +715,8 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -789,7 +757,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -798,8 +765,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin @@ -850,7 +815,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -859,8 +823,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin @@ -900,7 +862,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 |--conf spark.task.resource.gpu.amount=0.001 | |Comments: @@ -920,10 +881,8 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -1012,7 +971,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -1091,7 +1049,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -1168,7 +1125,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -1248,7 +1204,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -1537,7 +1492,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -1546,8 +1500,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.multiple")} [23.06.0, 23.02.1] |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin @@ -1585,7 +1537,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -1594,8 +1545,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- A newer RAPIDS Accelerator for Apache Spark plugin is available: | $pluginJarMvnURl | Version used in application is $jarVer. @@ -1630,7 +1579,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -1639,8 +1587,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin val rapidsJarsArr = Seq(s"rapids-4-spark_2.12-$latestRelease.jar") @@ -1886,7 +1832,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -1999,7 +1944,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -2068,7 +2012,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -2160,7 +2103,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.executor.instances' was not set. @@ -2174,10 +2116,8 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |- ${ProfilingAutoTunerConfigsProvider.commentForExperimentalConfig("spark.rapids.sql.incompatibleDateFormats.enabled")} @@ -2308,7 +2248,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- Setting 'spark.sql.adaptive.autoBroadcastJoinThreshold' > 100m could lead to performance\n regression. Should be set to a lower number. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -2665,7 +2604,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.shuffle.partitions' should be increased since spilling occurred in shuffle stages. - |- 'spark.sql.shuffle.partitions' was not set. |- Average JVM GC time is very high. Other Garbage Collectors can be used for better performance. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} @@ -2731,7 +2669,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.files.maxPartitionBytes=3669m - |--conf spark.sql.shuffle.partitions=200 | |Comments: |- 'spark.executor.memoryOverhead' was not set. @@ -2744,7 +2681,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. - |- 'spark.sql.shuffle.partitions' was not set. |- Average JVM GC time is very high. Other Garbage Collectors can be used for better performance. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- Shuffle skew exists (when task's Shuffle Read Size > 3 * Avg Stage-level size) in @@ -2802,7 +2738,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 |--conf spark.task.resource.gpu.amount=0.001 | |Comments: @@ -2821,10 +2756,8 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -2881,7 +2814,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 |--conf spark.task.resource.gpu.amount=0.001 | |Comments: @@ -2899,10 +2831,8 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -2959,7 +2889,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 |--conf spark.task.resource.gpu.amount=0.001 | |Comments: @@ -2977,10 +2906,8 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} @@ -3036,7 +2963,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.adaptive.autoBroadcastJoinThreshold=[FILL_IN_VALUE] |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m - |--conf spark.sql.shuffle.partitions=200 |--conf spark.task.resource.gpu.amount=0.001 | |Comments: @@ -3054,10 +2980,8 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. - |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.files.maxPartitionBytes' was not set. - |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala index c6891d636..a65ddd064 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala @@ -76,14 +76,18 @@ class QualificationAutoTunerSuite extends BaseAutoTunerSuite { assert(expectedResults.forall(autoTunerOutput.contains)) } - test("test AutoTuner for Qualification sets shuffle partitions to 200") { - val autoTuner = buildDefaultAutoTuner() + test("test AutoTuner for Qualification should not change shuffle partitions") { + // Set shuffle partitions to 100. The AutoTuner should recommend the same value + // because currently shuffle.partitions is one of the limitedLogicRecommendations. + // It will not be added to the recommendations because the value has not changed. + val autoTuner = buildDefaultAutoTuner( + defaultSparkProps ++ mutable.Map("spark.sql.shuffle.partitions" -> "100") + ) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = Seq( - "--conf spark.sql.shuffle.partitions=200", - "- 'spark.sql.shuffle.partitions' was not set." + "--conf spark.sql.shuffle.partitions" ) - assert(expectedResults.forall(autoTunerOutput.contains)) + assert(expectedResults.forall(t => !autoTunerOutput.contains(t))) } }