From 0e2a648a990ab9ed410d23f3fc77ae94292f48ec Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Mon, 9 May 2022 12:11:26 -0700 Subject: [PATCH 1/6] enabled optional confs and updated docs Signed-off-by: Raza Jafri --- docs/compatibility.md | 23 +++++++++---------- docs/configs.md | 20 ++++++++-------- .../getting-started-workload-qualification.md | 12 ++-------- docs/spark-profiling-tool.md | 1 - .../com/nvidia/spark/rapids/RapidsConf.scala | 20 ++++++++-------- 5 files changed, 33 insertions(+), 43 deletions(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index c1d3b6a575c..62c6bed4dca 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -63,8 +63,8 @@ conditions within the computation itself the result may not be the same each tim run. This is inherent in how the plugin speeds up the calculations and cannot be "fixed." If a query joins on a floating point value, which is not wise to do anyways, and the value is the result of a floating point aggregation then the join may fail to work properly with the plugin but would have -worked with plain Spark. Because of this most floating point aggregations are off by default but can -be enabled with the config +worked with plain Spark. As of 22.06 this is behavior is enabled by default but can be disabled with +the config [`spark.rapids.sql.variableFloatAgg.enabled`](configs.md#sql.variableFloatAgg.enabled). Additionally, some aggregations on floating point columns that contain `NaN` can produce results @@ -793,13 +793,12 @@ disabled on the GPU by default and require configuration options to be specified The GPU will use a different strategy from Java's BigDecimal to handle/store decimal values, which leads to restrictions: -* It is only available when `ansiMode` is on. * Float values cannot be larger than `1e18` or smaller than `-1e18` after conversion. * The results produced by GPU slightly differ from the default results of Spark. -To enable this operation on the GPU, set -[`spark.rapids.sql.castFloatToDecimal.enabled`](configs.md#sql.castFloatToDecimal.enabled) to `true` -and set `spark.sql.ansi.enabled` to `true`. +As of 22.06 this conf is enabled, to disable this operation on the GPU when using Spark 3.1.0 or +later, set +[`spark.rapids.sql.castFloatToDecimal.enabled`](configs.md#sql.castFloatToDecimal.enabled) to `false` ### Float to Integral Types @@ -809,9 +808,9 @@ Spark 3.1.0 the MIN and MAX values were floating-point values such as `Int.MaxVa starting with 3.1.0 these are now integral types such as `Int.MaxValue` so this has slightly affected the valid range of values and now differs slightly from the behavior on GPU in some cases. -To enable this operation on the GPU when using Spark 3.1.0 or later, set +As of 22.06 this conf is enabled, to disable this operation on the GPU when using Spark 3.1.0 or later, set [`spark.rapids.sql.castFloatToIntegralTypes.enabled`](configs.md#sql.castFloatToIntegralTypes.enabled) -to `true`. +to `false`. This configuration setting is ignored when using Spark versions prior to 3.1.0. @@ -821,8 +820,8 @@ The GPU will use different precision than Java's toString method when converting types to strings. The GPU uses a lowercase `e` prefix for an exponent while Spark uses uppercase `E`. As a result the computed string can differ from the default behavior in Spark. -To enable this operation on the GPU, set -[`spark.rapids.sql.castFloatToString.enabled`](configs.md#sql.castFloatToString.enabled) to `true`. +As of 22.06 this conf is enabled by default, to disable this operation on the GPU, set +[`spark.rapids.sql.castFloatToString.enabled`](configs.md#sql.castFloatToString.enabled) to `false`. ### String to Float @@ -835,8 +834,8 @@ default behavior in Apache Spark is to return `+Infinity` and `-Infinity`, respe Also, the GPU does not support casting from strings containing hex values. -To enable this operation on the GPU, set -[`spark.rapids.sql.castStringToFloat.enabled`](configs.md#sql.castStringToFloat.enabled) to `true`. +As of 22.06 this conf is enabled by default, to enable this operation on the GPU, set +[`spark.rapids.sql.castStringToFloat.enabled`](configs.md#sql.castStringToFloat.enabled) to `false`. ### String to Date diff --git a/docs/configs.md b/docs/configs.md index a69b47d8411..ae0b9f763ad 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -59,17 +59,17 @@ Name | Description | Default Value spark.rapids.shuffle.ucx.managementServerHost|The host to be used to start the management server|null spark.rapids.shuffle.ucx.useWakeup|When set to true, use UCX's event-based progress (epoll) in order to wake up the progress thread when needed, instead of a hot loop.|true spark.rapids.sql.batchSizeBytes|Set the target number of bytes for a GPU batch. Splits sizes for input data is covered by separate configs. The maximum setting is 2 GB to avoid exceeding the cudf row count limit of a column.|2147483647 -spark.rapids.sql.castDecimalToFloat.enabled|Casting from decimal to floating point types on the GPU returns results that have tiny difference compared to results returned from CPU.|false +spark.rapids.sql.castDecimalToFloat.enabled|Casting from decimal to floating point types on the GPU returns results that have tiny difference compared to results returned from CPU.|true spark.rapids.sql.castDecimalToString.enabled|When set to true, casting from decimal to string is supported on the GPU. The GPU does NOT produce exact same string as spark produces, but producing strings which are semantically equal. For instance, given input BigDecimal(123, -2), the GPU produces "12300", which spark produces "1.23E+4".|false -spark.rapids.sql.castFloatToDecimal.enabled|Casting from floating point types to decimal on the GPU returns results that have tiny difference compared to results returned from CPU.|false -spark.rapids.sql.castFloatToIntegralTypes.enabled|Casting from floating point types to integral types on the GPU supports a slightly different range of values when using Spark 3.1.0 or later. Refer to the CAST documentation for more details.|false -spark.rapids.sql.castFloatToString.enabled|Casting from floating point types to string on the GPU returns results that have a different precision than the default results of Spark.|false -spark.rapids.sql.castStringToFloat.enabled|When set to true, enables casting from strings to float types (float, double) on the GPU. Currently hex values aren't supported on the GPU. Also note that casting from string to float types on the GPU returns incorrect results when the string represents any number "1.7976931348623158E308" <= x < "1.7976931348623159E308" and "-1.7976931348623158E308" >= x > "-1.7976931348623159E308" in both these cases the GPU returns Double.MaxValue while CPU returns "+Infinity" and "-Infinity" respectively|false +spark.rapids.sql.castFloatToDecimal.enabled|Casting from floating point types to decimal on the GPU returns results that have tiny difference compared to results returned from CPU.|true +spark.rapids.sql.castFloatToIntegralTypes.enabled|Casting from floating point types to integral types on the GPU supports a slightly different range of values when using Spark 3.1.0 or later. Refer to the CAST documentation for more details.|true +spark.rapids.sql.castFloatToString.enabled|Casting from floating point types to string on the GPU returns results that have a different precision than the default results of Spark.|true +spark.rapids.sql.castStringToFloat.enabled|When set to true, enables casting from strings to float types (float, double) on the GPU. Currently hex values aren't supported on the GPU. Also note that casting from string to float types on the GPU returns incorrect results when the string represents any number "1.7976931348623158E308" <= x < "1.7976931348623159E308" and "-1.7976931348623158E308" >= x > "-1.7976931348623159E308" in both these cases the GPU returns Double.MaxValue while CPU returns "+Infinity" and "-Infinity" respectively|true spark.rapids.sql.castStringToTimestamp.enabled|When set to true, casting from string to timestamp is supported on the GPU. The GPU only supports a subset of formats when casting strings to timestamps. Refer to the CAST documentation for more details.|false spark.rapids.sql.concurrentGpuTasks|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.|1 spark.rapids.sql.csv.read.decimal.enabled|CSV reading is not 100% compatible when reading decimals.|false spark.rapids.sql.csv.read.double.enabled|CSV reading is not 100% compatible when reading doubles.|false -spark.rapids.sql.csv.read.float.enabled|CSV reading is not 100% compatible when reading floats.|false +spark.rapids.sql.csv.read.float.enabled|CSV reading is not 100% compatible when reading floats.|true spark.rapids.sql.decimalOverflowGuarantees|FOR TESTING ONLY. DO NOT USE IN PRODUCTION. Please see the decimal section of the compatibility documents for more information on this config.|true spark.rapids.sql.enabled|Enable (true) or disable (false) sql operations on the GPU|true spark.rapids.sql.explain|Explain why some parts of a query were not placed on a GPU or not. Possible values are ALL: print everything, NONE: print nothing, NOT_ON_GPU: print only parts of a query that did not go on the GPU|NONE @@ -98,10 +98,10 @@ Name | Description | Default Value spark.rapids.sql.hasExtendedYearValues|Spark 3.2.0+ extended parsing of years in dates and timestamps to support the full range of possible values. Prior to this it was limited to a positive 4 digit year. The Accelerator does not support the extended range yet. This config indicates if your data includes this extended range or not, or if you don't care about getting the correct values on values with the extended range.|true spark.rapids.sql.hasNans|Config to indicate if your data has NaN's. Cudf doesn't currently support NaN's properly so you can get corrupt data if you have NaN's in your data and it runs on the GPU.|true spark.rapids.sql.hashOptimizeSort.enabled|Whether sorts should be inserted after some hashed operations to improve output ordering. This can improve output file sizes when saving to columnar formats.|false -spark.rapids.sql.improvedFloatOps.enabled|For some floating point operations spark uses one way to compute the value and the underlying cudf implementation can use an improved algorithm. In some cases this can result in cudf producing an answer when spark overflows. Because this is not as compatible with spark, we have it disabled by default.|false +spark.rapids.sql.improvedFloatOps.enabled|For some floating point operations spark uses one way to compute the value and the underlying cudf implementation can use an improved algorithm. In some cases this can result in cudf producing an answer when spark overflows. Because this is not as compatible with spark, we have it disabled by default.|true spark.rapids.sql.improvedTimeOps.enabled|When set to true, some operators will avoid overflowing by converting epoch days directly to seconds without first converting to microseconds|false spark.rapids.sql.incompatibleDateFormats.enabled|When parsing strings as dates and timestamps in functions like unix_timestamp, some formats are fully supported on the GPU and some are unsupported and will fall back to the CPU. Some formats behave differently on the GPU than the CPU. Spark on the CPU interprets date formats with unsupported trailing characters as nulls, while Spark on the GPU will parse the date with invalid trailing characters. More detail can be found at [parsing strings as dates or timestamps](compatibility.md#parsing-strings-as-dates-or-timestamps).|false -spark.rapids.sql.incompatibleOps.enabled|For operations that work, but are not 100% compatible with the Spark equivalent set if they should be enabled by default or disabled by default.|false +spark.rapids.sql.incompatibleOps.enabled|For operations that work, but are not 100% compatible with the Spark equivalent set if they should be enabled by default or disabled by default.|true spark.rapids.sql.join.cross.enabled|When set to true cross joins are enabled on the GPU|true spark.rapids.sql.join.existence.enabled|When set to true existence joins are enabled on the GPU|true spark.rapids.sql.join.fullOuter.enabled|When set to true full outer joins are enabled on the GPU|true @@ -112,7 +112,7 @@ Name | Description | Default Value spark.rapids.sql.join.rightOuter.enabled|When set to true right outer joins are enabled on the GPU|true spark.rapids.sql.json.read.decimal.enabled|JSON reading is not 100% compatible when reading decimals.|false spark.rapids.sql.json.read.double.enabled|JSON reading is not 100% compatible when reading doubles.|false -spark.rapids.sql.json.read.float.enabled|JSON reading is not 100% compatible when reading floats.|false +spark.rapids.sql.json.read.float.enabled|JSON reading is not 100% compatible when reading floats.|true spark.rapids.sql.metrics.level|GPU plans can produce a lot more metrics than CPU plans do. In very large queries this can sometimes result in going over the max result size limit for the driver. Supported values include DEBUG which will enable all metrics supported and typically only needs to be enabled when debugging the plugin. MODERATE which should output enough metrics to understand how long each part of the query is taking and how much data is going to each part of the query. ESSENTIAL which disables most metrics except those Apache Spark CPU plans will also report or their equivalents.|MODERATE spark.rapids.sql.mode|Set the mode for the Rapids Accelerator. The supported modes are explainOnly and executeOnGPU. This config can not be changed at runtime, you must restart the application for it to take affect. The default mode is executeOnGPU, which means the RAPIDS Accelerator plugin convert the Spark operations and execute them on the GPU when possible. The explainOnly mode allows running queries on the CPU and the RAPIDS Accelerator will evaluate the queries as if it was going to run on the GPU. The explanations of what would have run on the GPU and why are output in log messages. When using explainOnly mode, the default explain output is ALL, this can be changed by setting spark.rapids.sql.explain. See that config for more details.|executeongpu spark.rapids.sql.python.gpu.enabled|This is an experimental feature and is likely to change in the future. Enable (true) or disable (false) support for scheduling Python Pandas UDFs with GPU resources. When enabled, pandas UDFs are assumed to share the same GPU that the RAPIDs accelerator uses and will honor the python GPU configs|false @@ -125,7 +125,7 @@ Name | Description | Default Value spark.rapids.sql.stableSort.enabled|Enable or disable stable sorting. Apache Spark's sorting is typically a stable sort, but sort stability cannot be guaranteed in distributed work loads because the order in which upstream data arrives to a task is not guaranteed. Sort stability then only matters when reading and sorting data from a file using a single task/partition. Because of limitations in the plugin when you enable stable sorting all of the data for a single task will be combined into a single batch before sorting. This currently disables spilling from GPU memory if the data size is too large.|false spark.rapids.sql.suppressPlanningFailure|Option to fallback an individual query to CPU if an unexpected condition prevents the query plan from being converted to a GPU-enabled one. Note this is different from a normal CPU fallback for a yet-to-be-supported Spark SQL feature. If this happens the error should be reported and investigated as a GitHub issue.|false spark.rapids.sql.udfCompiler.enabled|When set to true, Scala UDFs will be considered for compilation as Catalyst expressions|false -spark.rapids.sql.variableFloatAgg.enabled|Spark assumes that all operations produce the exact same result each time. This is not true for some floating point aggregations, which can produce slightly different results on the GPU as the aggregation is done in parallel. This can enable those operations if you know the query is only computing it once.|false +spark.rapids.sql.variableFloatAgg.enabled|Spark assumes that all operations produce the exact same result each time. This is not true for some floating point aggregations, which can produce slightly different results on the GPU as the aggregation is done in parallel. This can enable those operations if you know the query is only computing it once.|true spark.rapids.sql.window.range.byte.enabled|When the order-by column of a range based window is byte type and the range boundary calculated for a value has overflow, CPU and GPU will get the different results. When set to false disables the range window acceleration for the byte type order-by column|false spark.rapids.sql.window.range.int.enabled|When the order-by column of a range based window is int type and the range boundary calculated for a value has overflow, CPU and GPU will get the different results. When set to false disables the range window acceleration for the int type order-by column|true spark.rapids.sql.window.range.long.enabled|When the order-by column of a range based window is long type and the range boundary calculated for a value has overflow, CPU and GPU will get the different results. When set to false disables the range window acceleration for the long type order-by column|true diff --git a/docs/get-started/getting-started-workload-qualification.md b/docs/get-started/getting-started-workload-qualification.md index a89934d40ec..d99586dcde0 100644 --- a/docs/get-started/getting-started-workload-qualification.md +++ b/docs/get-started/getting-started-workload-qualification.md @@ -101,15 +101,11 @@ the driver logs with `spark.rapids.sql.explain=all`. [configuration documentation](../configs.md) for details of RAPIDS Accelerator parameters. - For example, if your jobs have `double`, `float` and `decimal` operators together with some Scala + For example, if your jobs have `decimal` operators together with some Scala UDFs, you can set the following parameters: ```scala - spark.conf.set("spark.rapids.sql.incompatibleOps.enabled", true) - spark.conf.set("spark.rapids.sql.variableFloatAgg.enabled", true) spark.conf.set("spark.rapids.sql.decimalType.enabled", true) - spark.conf.set("spark.rapids.sql.castFloatToDecimal.enabled",true) - spark.conf.set("spark.rapids.sql.castDecimalToFloat.enabled",true) spark.conf.set("spark.rapids.sql.udfCompiler.enabled",true) ``` @@ -220,15 +216,11 @@ Throws: the meaning and risk of above parameters before enabling it. Please refer to the [configuration documentation](../configs.md) for details of RAPIDS Accelerator parameters. - For example, if your jobs have `double`, `float` and `decimal` operators together with some Scala + For example, if your jobs have `decimal` operators together with some Scala UDFs, you can set the following parameters: ```scala - spark.conf.set("spark.rapids.sql.incompatibleOps.enabled", true) - spark.conf.set("spark.rapids.sql.variableFloatAgg.enabled", true) spark.conf.set("spark.rapids.sql.decimalType.enabled", true) - spark.conf.set("spark.rapids.sql.castFloatToDecimal.enabled",true) - spark.conf.set("spark.rapids.sql.castDecimalToFloat.enabled",true) spark.conf.set("spark.rapids.sql.udfCompiler.enabled",true) ``` diff --git a/docs/spark-profiling-tool.md b/docs/spark-profiling-tool.md index 3143a8cff16..986036533c1 100644 --- a/docs/spark-profiling-tool.md +++ b/docs/spark-profiling-tool.md @@ -243,7 +243,6 @@ Compare Rapids Properties which are set explicitly: |spark.rapids.sql.explain |null |NOT_ON_GPU| |spark.rapids.sql.hasNans |null |FALSE | |spark.rapids.sql.incompatibleOps.enabled |null |true | -|spark.rapids.sql.variableFloatAgg.enabled |null |TRUE | +-------------------------------------------+----------+----------+ ``` diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 05afab07bfe..8a76f13b638 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -559,7 +559,7 @@ object RapidsConf { .doc("For operations that work, but are not 100% compatible with the Spark equivalent " + "set if they should be enabled by default or disabled by default.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val INCOMPATIBLE_DATE_FORMATS = conf("spark.rapids.sql.incompatibleDateFormats.enabled") .doc("When parsing strings as dates and timestamps in functions like unix_timestamp, some " + @@ -578,7 +578,7 @@ object RapidsConf { "In some cases this can result in cudf producing an answer when spark overflows. " + "Because this is not as compatible with spark, we have it disabled by default.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val HAS_NANS = conf("spark.rapids.sql.hasNans") .doc("Config to indicate if your data has NaN's. Cudf doesn't " + @@ -599,7 +599,7 @@ object RapidsConf { "different results on the GPU as the aggregation is done in parallel. This can enable " + "those operations if you know the query is only computing it once.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val ENABLE_REPLACE_SORTMERGEJOIN = conf("spark.rapids.sql.replaceSortMergeJoin.enabled") .doc("Allow replacing sortMergeJoin with HashJoin") @@ -616,13 +616,13 @@ object RapidsConf { .doc("Casting from floating point types to decimal on the GPU returns results that have " + "tiny difference compared to results returned from CPU.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val ENABLE_CAST_FLOAT_TO_STRING = conf("spark.rapids.sql.castFloatToString.enabled") .doc("Casting from floating point types to string on the GPU returns results that have " + "a different precision than the default results of Spark.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val ENABLE_CAST_FLOAT_TO_INTEGRAL_TYPES = conf("spark.rapids.sql.castFloatToIntegralTypes.enabled") @@ -630,13 +630,13 @@ object RapidsConf { "slightly different range of values when using Spark 3.1.0 or later. Refer to the CAST " + "documentation for more details.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val ENABLE_CAST_DECIMAL_TO_FLOAT = conf("spark.rapids.sql.castDecimalToFloat.enabled") .doc("Casting from decimal to floating point types on the GPU returns results that have " + "tiny difference compared to results returned from CPU.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val ENABLE_CAST_STRING_TO_FLOAT = conf("spark.rapids.sql.castStringToFloat.enabled") .doc("When set to true, enables casting from strings to float types (float, double) " + @@ -647,7 +647,7 @@ object RapidsConf { "the GPU returns Double.MaxValue while CPU returns \"+Infinity\" and \"-Infinity\" " + "respectively") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val ENABLE_CAST_STRING_TO_TIMESTAMP = conf("spark.rapids.sql.castStringToTimestamp.enabled") .doc("When set to true, casting from string to timestamp is supported on the GPU. The GPU " + @@ -888,7 +888,7 @@ object RapidsConf { val ENABLE_READ_CSV_FLOATS = conf("spark.rapids.sql.csv.read.float.enabled") .doc("CSV reading is not 100% compatible when reading floats.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val ENABLE_READ_CSV_DOUBLES = conf("spark.rapids.sql.csv.read.double.enabled") .doc("CSV reading is not 100% compatible when reading doubles.") @@ -914,7 +914,7 @@ object RapidsConf { val ENABLE_READ_JSON_FLOATS = conf("spark.rapids.sql.json.read.float.enabled") .doc("JSON reading is not 100% compatible when reading floats.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val ENABLE_READ_JSON_DOUBLES = conf("spark.rapids.sql.json.read.double.enabled") .doc("JSON reading is not 100% compatible when reading doubles.") From c653c59254beae9392f8f13c1e03a034023d4847 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Thu, 12 May 2022 13:11:21 -0700 Subject: [PATCH 2/6] addressed review comments Signed-off-by: Raza Jafri --- .../get-started/getting-started-workload-qualification.md | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/docs/get-started/getting-started-workload-qualification.md b/docs/get-started/getting-started-workload-qualification.md index d99586dcde0..3837190bc77 100644 --- a/docs/get-started/getting-started-workload-qualification.md +++ b/docs/get-started/getting-started-workload-qualification.md @@ -101,11 +101,9 @@ the driver logs with `spark.rapids.sql.explain=all`. [configuration documentation](../configs.md) for details of RAPIDS Accelerator parameters. - For example, if your jobs have `decimal` operators together with some Scala - UDFs, you can set the following parameters: + For example, if your jobs Scala UDFs, you can set the following parameters: ```scala - spark.conf.set("spark.rapids.sql.decimalType.enabled", true) spark.conf.set("spark.rapids.sql.udfCompiler.enabled",true) ``` @@ -216,11 +214,9 @@ Throws: the meaning and risk of above parameters before enabling it. Please refer to the [configuration documentation](../configs.md) for details of RAPIDS Accelerator parameters. - For example, if your jobs have `decimal` operators together with some Scala - UDFs, you can set the following parameters: + For example, if your jobs have Scala UDFs, you can set the following parameters: ```scala - spark.conf.set("spark.rapids.sql.decimalType.enabled", true) spark.conf.set("spark.rapids.sql.udfCompiler.enabled",true) ``` From d78c7156c5b669d5e272d6964be5e905db861743 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 13 May 2022 12:10:23 -0700 Subject: [PATCH 3/6] removed mention of variables from tunin-guide Signed-off-by: Raza Jafri --- docs/tuning-guide.md | 24 ------------------------ 1 file changed, 24 deletions(-) diff --git a/docs/tuning-guide.md b/docs/tuning-guide.md index d34cb792b5c..daef39a4cf9 100644 --- a/docs/tuning-guide.md +++ b/docs/tuning-guide.md @@ -278,30 +278,6 @@ from the main [columnar batch size](#columnar-batch-size) setting. Some transco load CSV files then write Parquet files) need to lower this setting when using large task input partition sizes to avoid GPU out of memory errors. -## Enable Incompatible Operations -Configuration key: -[`spark.rapids.sql.incompatibleOps.enabled`](configs.md#sql.incompatibleOps.enabled) - -Default value: `false` - -There are several operators/expressions that are not 100% compatible with the CPU version. These -incompatibilities are documented [here](compatibility.md) and in the -[configuration documentation](./configs.md). Many of these incompatibilities are around corner -cases that most queries do not encounter, or that would not result in any meaningful difference -to the resulting output. By enabling these operations either individually or with the -`spark.rapids.sql.incompatibleOps.enabled` config it can greatly improve performance of your -queries. Over time, we expect the number of incompatible operators to reduce. - -If you want to understand if an operation is or is not on the GPU and why see second on -[explain in the FAQ](FAQ.md#explain) - -The following configs all enable different types of incompatible operations that can improve -performance. -- [`spark.rapids.sql.variableFloatAgg.enabled`](configs.md#sql.variableFloatAgg.enabled) -- [`spark.rapids.sql.hasNans`](configs.md#sql.hasNans) -- [`spark.rapids.sql.castFloatToString.enabled`](configs.md#sql.castFloatToString.enabled) -- [`spark.rapids.sql.castStringToFloat.enabled`](configs.md#sql.castStringToFloat.enabled) - ## Metrics Custom Spark SQL Metrics are available which can help identify performance bottlenecks in a query. From 1f6be873ddf47854c8e3a40a323d2a0b6c6bfd88 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Mon, 23 May 2022 11:34:47 -0700 Subject: [PATCH 4/6] added back incompatible operations section Signed-off-by: Raza Jafri --- docs/tuning-guide.md | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/tuning-guide.md b/docs/tuning-guide.md index daef39a4cf9..0f6392539d6 100644 --- a/docs/tuning-guide.md +++ b/docs/tuning-guide.md @@ -278,6 +278,30 @@ from the main [columnar batch size](#columnar-batch-size) setting. Some transco load CSV files then write Parquet files) need to lower this setting when using large task input partition sizes to avoid GPU out of memory errors. +## Enable Incompatible Operations +Configuration key: +[`spark.rapids.sql.incompatibleOps.enabled`](configs.md#sql.incompatibleOps.enabled) + +Default value: `true` + +There are several operators/expressions that are not 100% compatible with the CPU version. These +incompatibilities are documented [here](compatibility.md) and in the +[configuration documentation](./configs.md). Many of these incompatibilities are around corner +cases that most queries do not encounter, or that would not result in any meaningful difference +to the resulting output. By enabling these operations either individually or with the +`spark.rapids.sql.incompatibleOps.enabled` config it can greatly improve performance of your +queries. Over time, we expect the number of incompatible operators to reduce. + +If you want to understand if an operation is or is not on the GPU and why see second on +[explain in the FAQ](FAQ.md#explain) + +The following configs all enable different types of incompatible operations that can improve +performance. +- [`spark.rapids.sql.variableFloatAgg.enabled`](configs.md#sql.variableFloatAgg.enabled) +- [`spark.rapids.sql.hasNans`](configs.md#sql.hasNans) +- [`spark.rapids.sql.castFloatToString.enabled`](configs.md#sql.castFloatToString.enabled) +- [`spark.rapids.sql.castStringToFloat.enabled`](configs.md#sql.castStringToFloat.enabled) + ## Metrics Custom Spark SQL Metrics are available which can help identify performance bottlenecks in a query. From fbb207ff5ca2906e50f584f9c47029f86eab8285 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 25 May 2022 17:22:28 -0700 Subject: [PATCH 5/6] adding info about setting hasNans to false Signed-off-by: Raza Jafri --- docs/tuning-guide.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/tuning-guide.md b/docs/tuning-guide.md index 0f6392539d6..436bc0edccf 100644 --- a/docs/tuning-guide.md +++ b/docs/tuning-guide.md @@ -302,6 +302,9 @@ performance. - [`spark.rapids.sql.castFloatToString.enabled`](configs.md#sql.castFloatToString.enabled) - [`spark.rapids.sql.castStringToFloat.enabled`](configs.md#sql.castStringToFloat.enabled) +NOTE: If your data doesn't contain NaNs set [`spark.rapids.sql.hasNans`](configs.md#sql.hasNans) to `false` to get the +benefit of running on the GPU + ## Metrics Custom Spark SQL Metrics are available which can help identify performance bottlenecks in a query. From fab3fef49953cd79d4825a33e448b574e669788e Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 25 May 2022 17:39:11 -0700 Subject: [PATCH 6/6] addressed review comments Signed-off-by: Raza Jafri --- docs/tuning-guide.md | 29 +++-------------------------- 1 file changed, 3 insertions(+), 26 deletions(-) diff --git a/docs/tuning-guide.md b/docs/tuning-guide.md index 436bc0edccf..26d68e358f9 100644 --- a/docs/tuning-guide.md +++ b/docs/tuning-guide.md @@ -278,32 +278,9 @@ from the main [columnar batch size](#columnar-batch-size) setting. Some transco load CSV files then write Parquet files) need to lower this setting when using large task input partition sizes to avoid GPU out of memory errors. -## Enable Incompatible Operations -Configuration key: -[`spark.rapids.sql.incompatibleOps.enabled`](configs.md#sql.incompatibleOps.enabled) - -Default value: `true` - -There are several operators/expressions that are not 100% compatible with the CPU version. These -incompatibilities are documented [here](compatibility.md) and in the -[configuration documentation](./configs.md). Many of these incompatibilities are around corner -cases that most queries do not encounter, or that would not result in any meaningful difference -to the resulting output. By enabling these operations either individually or with the -`spark.rapids.sql.incompatibleOps.enabled` config it can greatly improve performance of your -queries. Over time, we expect the number of incompatible operators to reduce. - -If you want to understand if an operation is or is not on the GPU and why see second on -[explain in the FAQ](FAQ.md#explain) - -The following configs all enable different types of incompatible operations that can improve -performance. -- [`spark.rapids.sql.variableFloatAgg.enabled`](configs.md#sql.variableFloatAgg.enabled) -- [`spark.rapids.sql.hasNans`](configs.md#sql.hasNans) -- [`spark.rapids.sql.castFloatToString.enabled`](configs.md#sql.castFloatToString.enabled) -- [`spark.rapids.sql.castStringToFloat.enabled`](configs.md#sql.castStringToFloat.enabled) - -NOTE: If your data doesn't contain NaNs set [`spark.rapids.sql.hasNans`](configs.md#sql.hasNans) to `false` to get the -benefit of running on the GPU +## Set hasNans flag to False +If your data has float values but doesn't contain NaNs set [`spark.rapids.sql.hasNans`](configs.md#sql.hasNans) to +`false` to get the benefit of running on the GPU ## Metrics