From c24dcfe712faefd7d7feb3c0a5550724f5e053a9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 30 Nov 2021 17:05:24 -0700 Subject: [PATCH 01/52] Make EOL anchor behavior match CPU for strings ending with newline character (#4239) Signed-off-by: Andy Grove --- .../com/nvidia/spark/rapids/RegexParser.scala | 12 ++++++- .../RegularExpressionTranspilerSuite.scala | 35 +++++++++---------- 2 files changed, 27 insertions(+), 20 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index c60e8db3524..86ba30d9c83 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -444,7 +444,17 @@ class CudfRegexTranspiler(replace: Boolean) { // this is a bit extreme and it would be good to replace with finer-grained // rules throw new RegexUnsupportedException("regexp_replace on GPU does not support ^ or $") - + case '$' => + RegexSequence(ListBuffer( + RegexRepetition( + RegexCharacterClass(negated = false, + characters = ListBuffer(RegexChar('\r'))), + SimpleQuantifier('?')), + RegexRepetition( + RegexCharacterClass(negated = false, + characters = ListBuffer(RegexChar('\n'))), + SimpleQuantifier('?')), + RegexChar('$'))) case _ => regex } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala index 9bd859f6562..240ba7d6eab 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala @@ -15,7 +15,7 @@ */ package com.nvidia.spark.rapids -import java.util.regex.Pattern +import java.util.regex.{Matcher, Pattern} import scala.collection.mutable.ListBuffer import scala.util.{Random, Try} @@ -132,13 +132,16 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { assertUnsupported(pattern, "nothing to repeat")) } - ignore("known issue - multiline difference between CPU and GPU") { - // see https://github.com/rapidsai/cudf/issues/9620 + test("end of line anchor with strings ending in valid newline") { val pattern = "2$" - // this matches "2" but not "2\n" on the GPU assertCpuGpuMatchesRegexpFind(Seq(pattern), Seq("2", "2\n", "2\r", "2\r\n")) } + test("end of line anchor with strings ending in invalid newline") { + val pattern = "2$" + assertCpuGpuMatchesRegexpFind(Seq(pattern), Seq("2\n\r")) + } + test("dot matches CR on GPU but not on CPU") { // see https://github.com/rapidsai/cudf/issues/9619 val pattern = "1." @@ -187,8 +190,10 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { ")" + "$" // end of line - // input and output should be identical - doTranspileTest(VALID_FLOAT_REGEX, VALID_FLOAT_REGEX) + // input and output should be identical except for '$' being replaced with '[\r]?[\n]?$' + doTranspileTest(VALID_FLOAT_REGEX, + VALID_FLOAT_REGEX.replaceAll("\\$", + Matcher.quoteReplacement("[\r]?[\n]?$"))) } test("transpile complex regex 2") { @@ -197,9 +202,11 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { "(.[1-9]*(?:0)?[1-9]+)?(.0*[1-9]+)?(?:.0*)?$" // input and output should be identical except for `.` being replaced with `[^\r\n]` + // and '$' being replaced with '[\r]?[\n]?$' doTranspileTest(TIMESTAMP_TRUNCATE_REGEX, - TIMESTAMP_TRUNCATE_REGEX.replaceAll("\\.", "[^\r\n]")) - + TIMESTAMP_TRUNCATE_REGEX + .replaceAll("\\.", "[^\r\n]") + .replaceAll("\\$", Matcher.quoteReplacement("[\r]?[\n]?$"))) } test("compare CPU and GPU: character range including unescaped + and -") { @@ -257,7 +264,6 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { // LF has been excluded due to known issues val chars = (0x00 to 0x7F) .map(_.toChar) - .filterNot(_ == '\n') doFuzzTest(Some(chars.mkString), replace = true) } @@ -272,8 +278,7 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { options = FuzzerOptions(validChars, maxStringLen = 12)) val data = Range(0, 1000) - // remove trailing newlines as workaround for https://github.com/rapidsai/cudf/issues/9620 - .map(_ => removeTrailingNewlines(r.nextString())) + .map(_ => r.nextString()) // generate patterns that are valid on both CPU and GPU val patterns = ListBuffer[String]() @@ -291,14 +296,6 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { } } - private def removeTrailingNewlines(input: String): String = { - var s = input - while (s.endsWith("\r") || s.endsWith("\n")) { - s = s.substring(0, s.length - 1) - } - s - } - private def assertCpuGpuMatchesRegexpFind(javaPatterns: Seq[String], input: Seq[String]) = { for (javaPattern <- javaPatterns) { val cpu = cpuContains(javaPattern, input) From 2766002e78e259c32371600855a098b9c75642f1 Mon Sep 17 00:00:00 2001 From: zhanga5 Date: Wed, 1 Dec 2021 13:42:00 +0800 Subject: [PATCH 02/52] Add arguments support to 'databricks/run-tests.py' [databricks] (#4222) * Fix flake8 reported issues e.g, 'flake8 --max-line-length=120 jenkins/databricks/params.py jenkins/databricks/run-tests.py' Signed-off-by: Alex Zhang * Add arguments support to 'databricks/run-tests.py' Signed-off-by: Alex Zhang * Add common 'usage()' per review comments and some minor refinement Signed-off-by: Alex Zhang --- jenkins/databricks/params.py | 48 ++++++++++++++++++++++++++------- jenkins/databricks/run-tests.py | 43 ++++++++++++++++------------- 2 files changed, 63 insertions(+), 28 deletions(-) diff --git a/jenkins/databricks/params.py b/jenkins/databricks/params.py index 11bcc67d86c..449bd8dceb6 100644 --- a/jenkins/databricks/params.py +++ b/jenkins/databricks/params.py @@ -11,9 +11,10 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +"""Parse input parameters.""" -import sys import getopt +import sys workspace = 'https://dbc-9ff9942e-a9c4.cloud.databricks.com' token = '' @@ -27,22 +28,50 @@ clusterid = '' build_profiles = 'databricks,!snapshot-shims' jar_path = '' -# `spark_conf` can take comma seperated mutiple spark configurations, e.g., spark.foo=1,spark.bar=2,...' +# `spark_conf` can take comma seperated multiple spark configurations, e.g., spark.foo=1,spark.bar=2,...' spark_conf = '' + +def usage(): + """Define usage.""" + print('Usage: ' + sys.argv[0] + + ' -s ' + ' -t ' + ' -c ' + ' -p ' + ' -l ' + ' -d ' + ' -z ' + ' -v ' + ' -b ' + ' -j ' + ' -n ' + ' -f ' + ' -i ') + + try: - opts, args = getopt.getopt(sys.argv[1:], 'hw:t:c:p:l:d:z:m:v:b:j:f:i:', - ['workspace=', 'token=', 'clusterid=', 'private=', 'localscript=', 'dest=', 'sparktgz=', 'basesparkpomversion=', 'buildprofiles=', 'jarpath', 'sparkconf', 'sparkinstallver=']) + opts, script_args = getopt.getopt(sys.argv[1:], 'hw:t:c:p:l:d:z:m:v:b:j:f:i:', + ['workspace=', + 'token=', + 'clusterid=', + 'private=', + 'localscript=', + 'dest=', + 'sparktgz=', + 'basesparkpomversion=', + 'buildprofiles=', + 'jarpath', + 'sparkconf', + 'sparkinstallver=']) except getopt.GetoptError: - print( - 'run-tests.py -s -t -c -p -l -d -z -v -b -j -f -i ') + usage() sys.exit(2) for opt, arg in opts: if opt == '-h': - print( - 'run-tests.py -s -t -c -p -n -l -d , -z -v -b -f -i ') - sys.exit() + usage() + sys.exit(1) elif opt in ('-w', '--workspace'): workspace = arg elif opt in ('-t', '--token'): @@ -73,6 +102,7 @@ print('-p is ' + private_key_file) print('-l is ' + local_script) print('-d is ' + script_dest) +print('script_args is ' + ' '.join(script_args)) print('-z is ' + source_tgz) print('-v is ' + base_spark_pom_version) print('-j is ' + jar_path) diff --git a/jenkins/databricks/run-tests.py b/jenkins/databricks/run-tests.py index cee3f777840..9d2b8a92431 100644 --- a/jenkins/databricks/run-tests.py +++ b/jenkins/databricks/run-tests.py @@ -11,33 +11,38 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -import json -import requests -import sys -import getopt -import time -import os +"""Upload & run test script on Databricks cluster.""" + import subprocess +import sys + from clusterutils import ClusterUtils + import params def main(): + """Define main function.""" + master_addr = ClusterUtils.cluster_get_master_addr(params.workspace, params.clusterid, params.token) + if master_addr is None: + print("Error, didn't get master address") + sys.exit(1) + print("Master node address is: %s" % master_addr) - master_addr = ClusterUtils.cluster_get_master_addr(params.workspace, params.clusterid, params.token) - if master_addr is None: - print("Error, didn't get master address") - sys.exit(1) - print("Master node address is: %s" % master_addr) + print("Copying script") + rsync_command = "rsync -I -Pave \"ssh -o StrictHostKeyChecking=no -o UserKnownHostsFile=/dev/null -p 2200 -i %s\"" \ + " %s ubuntu@%s:%s" % (params.private_key_file, params.local_script, master_addr, params.script_dest) + print("rsync command: %s" % rsync_command) + subprocess.check_call(rsync_command, shell=True) - print("Copying script") - rsync_command = "rsync -I -Pave \"ssh -o StrictHostKeyChecking=no -o UserKnownHostsFile=/dev/null -p 2200 -i %s\" %s ubuntu@%s:%s" % (params.private_key_file, params.local_script, master_addr, params.script_dest) - print("rsync command: %s" % rsync_command) - subprocess.check_call(rsync_command, shell = True) + ssh_command = "ssh -o StrictHostKeyChecking=no -o UserKnownHostsFile=/dev/null ubuntu@%s -p 2200 -i %s " \ + "'LOCAL_JAR_PATH=%s SPARK_CONF=%s BASE_SPARK_VER=%s bash %s %s 2>&1 | tee testout; " \ + "if [ ${PIPESTATUS[0]} -ne 0 ]; then false; else true; fi'" % \ + (master_addr, params.private_key_file, params.jar_path, params.spark_conf, params.base_spark_pom_version, + params.script_dest, ' '.join(params.script_args)) + print("ssh command: %s" % ssh_command) + subprocess.check_call(ssh_command, shell=True) - ssh_command = "ssh -o StrictHostKeyChecking=no -o UserKnownHostsFile=/dev/null ubuntu@%s -p 2200 -i %s 'LOCAL_JAR_PATH=%s SPARK_CONF=%s BASE_SPARK_VER=%s bash %s 2>&1 | tee testout; if [ ${PIPESTATUS[0]} -ne 0 ]; then false; else true; fi'" % (master_addr, params.private_key_file, params.jar_path, params.spark_conf, params.base_spark_pom_version, params.script_dest) - print("ssh command: %s" % ssh_command) - subprocess.check_call(ssh_command, shell = True) if __name__ == '__main__': - main() + main() From 91d338377adaa7300ab8719d26b7a9b9ee50b611 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 Dec 2021 00:02:26 -0700 Subject: [PATCH 03/52] Disable failing CastOpSuite test (#4247) * disable failing CastOpSuite test Signed-off-by: Andy Grove --- tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala index ad373a2423f..de3f7acc3d7 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala @@ -126,7 +126,8 @@ class CastOpSuite extends GpuExpressionTestSuite { testCastStringTo(DataTypes.FloatType, generateRandomStrings(Some(NUMERIC_CHARS))) } - test("Cast from string to float using hand-picked values") { + // https://github.com/NVIDIA/spark-rapids/issues/4246 + ignore("Cast from string to float using hand-picked values") { testCastStringTo(DataTypes.FloatType, Seq(".", "e", "Infinity", "+Infinity", "-Infinity", "+nAn", "-naN", "Nan", "5f", "1.2f", "\riNf", null)) } From daa1221554ccd52f82de79ccc0cd0bf56e71d339 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 Dec 2021 12:29:51 -0700 Subject: [PATCH 04/52] Skip test_regexp_replace_null_pattern_fallback on Databricks [databricks] (#4251) * Disable test_regexp_replace_null_pattern_fallback when running against Databricks Signed-off-by: Andy Grove * update tests to use skipif instead of xfail --- integration_tests/src/main/python/string_test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/integration_tests/src/main/python/string_test.py b/integration_tests/src/main/python/string_test.py index f89d0a7f96d..48032f14b20 100644 --- a/integration_tests/src/main/python/string_test.py +++ b/integration_tests/src/main/python/string_test.py @@ -220,7 +220,7 @@ def test_concat_ws_sql_col_sep(): 'concat_ws(c, b, a, cast(null as string)) from concat_ws_table') -@pytest.mark.xfail(condition=is_databricks_runtime(), +@pytest.mark.skipif(is_databricks_runtime(), reason='Databricks optimizes out concat_ws call in this case') @allow_non_gpu('ProjectExec', 'Alias', 'ConcatWs') def test_concat_ws_sql_col_sep_only_sep_specified(): @@ -483,11 +483,14 @@ def test_regexp_replace(): 'regexp_replace(a, "a|b|c", "A")'), conf={'spark.rapids.sql.expression.RegExpReplace': 'true'}) +@pytest.mark.skipif(is_databricks_runtime(), + reason='Databricks optimizes out regexp_replace call in this case') @allow_non_gpu('ProjectExec', 'RegExpReplace') def test_regexp_replace_null_pattern_fallback(): gen = mk_str_gen('[abcd]{0,3}') - # Spark translates `NULL` to `CAST(NULL as STRING)` and we only support + # Apache Spark translates `NULL` to `CAST(NULL as STRING)` and we only support # literal expressions for the regex pattern + # Databricks Spark replaces the whole regexp_replace expression with a literal null assert_gpu_fallback_collect( lambda spark: unary_op_df(spark, gen).selectExpr( 'regexp_replace(a, NULL, "A")'), From d95b043c5d70fcf6d87775d2d5a699775ec6d34d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 Dec 2021 14:49:53 -0700 Subject: [PATCH 05/52] Implement duplicate key handling for GpuCreateMap (#4007) * Implement duplicate key handling for GpuCreateMap Signed-off-by: Andy Grove * code formatting * close temp resources Signed-off-by: Andy Grove * simplify code * address PR feedback * remove unused import * add integration tests for duplicate key exception cases --- docs/configs.md | 1 - integration_tests/src/main/python/map_test.py | 44 +++++++++++++------ .../com/nvidia/spark/rapids/GpuMapUtils.scala | 2 +- .../com/nvidia/spark/rapids/RapidsConf.scala | 10 ----- .../com/nvidia/spark/rapids/TypeChecks.scala | 23 ---------- .../spark/sql/rapids/complexTypeCreator.scala | 28 ++++++++++-- 6 files changed, 55 insertions(+), 53 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index c099987fed6..435e08d2161 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -67,7 +67,6 @@ Name | Description | Default Value 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.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.createMap.enabled|The GPU-enabled version of the `CreateMap` expression (`map` SQL function) does not detect duplicate keys in all cases and does not guarantee which key wins if there are duplicates. When this config is set to true, `CreateMap` will be enabled to run on the GPU even when there might be duplicate keys.|false spark.rapids.sql.csv.read.bool.enabled|Parsing an invalid CSV boolean value produces true instead of null|false spark.rapids.sql.csv.read.byte.enabled|Parsing CSV bytes is much more lenient and will return 0 for some malformed values instead of null|false spark.rapids.sql.csv.read.date.enabled|Parsing invalid CSV dates produces different results from Spark|false diff --git a/integration_tests/src/main/python/map_test.py b/integration_tests/src/main/python/map_test.py index 77c65a0fc2a..72aea6fbd2e 100644 --- a/integration_tests/src/main/python/map_test.py +++ b/integration_tests/src/main/python/map_test.py @@ -96,27 +96,43 @@ def test_map_expr_multiple_pairs(): assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, data_gen).selectExpr( 'map("key1", b, "key2", a) as m1', - 'map(a, b, b, a) as m2'), - conf={'spark.rapids.sql.createMap.enabled':True}) + 'map(a, b, b, a) as m2')) -@allow_non_gpu('ProjectExec,Alias,CreateMap,Literal') -def test_map_expr_dupe_keys_fallback(): +def test_map_expr_expr_keys_dupe_last_win(): data_gen = [('a', StringGen(nullable=False)), ('b', StringGen(nullable=False))] - assert_gpu_fallback_collect( + assert_gpu_and_cpu_are_equal_collect( + lambda spark : gen_df(spark, data_gen).selectExpr( + 'map(a, b, a, b) as m2'), + conf={'spark.sql.mapKeyDedupPolicy':'LAST_WIN'}) + +def test_map_expr_expr_keys_dupe_exception(): + data_gen = [('a', StringGen(nullable=False)), ('b', StringGen(nullable=False))] + assert_gpu_and_cpu_error( + lambda spark : gen_df(spark, data_gen).selectExpr( + 'map(a, b, a, b) as m2').collect(), + conf={'spark.sql.mapKeyDedupPolicy':'EXCEPTION'}, + error_message = "Duplicate map key") + +def test_map_expr_literal_keys_dupe_last_win(): + data_gen = [('a', StringGen(nullable=False)), ('b', StringGen(nullable=False))] + assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, data_gen).selectExpr( 'map("key1", b, "key1", a) as m1'), - 'ProjectExec', - conf={'spark.rapids.sql.createMap.enabled':True, - 'spark.sql.mapKeyDedupPolicy':'LAST_WIN'}) + conf={'spark.sql.mapKeyDedupPolicy':'LAST_WIN'}) -@allow_non_gpu('ProjectExec,Alias,CreateMap,Literal') -def test_map_expr_multi_non_literal_keys_fallback(): +def test_map_expr_literal_keys_dupe_exception(): data_gen = [('a', StringGen(nullable=False)), ('b', StringGen(nullable=False))] - assert_gpu_fallback_collect( + assert_gpu_and_cpu_error( + lambda spark : gen_df(spark, data_gen).selectExpr( + 'map("key1", b, "key1", a) as m1').collect(), + conf={'spark.sql.mapKeyDedupPolicy':'EXCEPTION'}, + error_message = "Duplicate map key") + +def test_map_expr_multi_non_literal_keys(): + data_gen = [('a', StringGen(nullable=False)), ('b', StringGen(nullable=False))] + assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, data_gen).selectExpr( - 'map(a, b, b, a) as m1'), - 'ProjectExec', - conf={'spark.rapids.sql.createMap.enabled':False}) + 'map(a, b, b, a) as m1')) def test_map_scalar_project(): assert_gpu_and_cpu_are_equal_collect( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMapUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMapUtils.scala index 098f0a69396..fa1052e9fcc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMapUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMapUtils.scala @@ -127,7 +127,7 @@ object GpuMapUtils extends Arm { } // Copied from Spark org.apache.spark.sql.errors.QueryExecutionErrors - private[this] def duplicateMapKeyFoundError: Throwable = { + def duplicateMapKeyFoundError: Throwable = { new RuntimeException(s"Duplicate map key was found, please check the input " + "data. If you want to remove the duplicated keys, you can set " + s"${SQLConf.MAP_KEY_DEDUP_POLICY.key} to ${SQLConf.MapKeyDedupPolicy.LAST_WIN} so that " + 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 12d9c07965f..169840f4b6e 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 @@ -642,14 +642,6 @@ object RapidsConf { .booleanConf .createWithDefault(false) - val ENABLE_CREATE_MAP = conf("spark.rapids.sql.createMap.enabled") - .doc("The GPU-enabled version of the `CreateMap` expression (`map` SQL function) does not " + - "detect duplicate keys in all cases and does not guarantee which key wins if there are " + - "duplicates. When this config is set to true, `CreateMap` will be enabled to run on the " + - "GPU even when there might be duplicate keys.") - .booleanConf - .createWithDefault(false) - val ENABLE_INNER_JOIN = conf("spark.rapids.sql.join.inner.enabled") .doc("When set to true inner joins are enabled on the GPU") .booleanConf @@ -1589,8 +1581,6 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isProjectAstEnabled: Boolean = get(ENABLE_PROJECT_AST) - lazy val isCreateMapEnabled: Boolean = get(ENABLE_CREATE_MAP) - lazy val isParquetEnabled: Boolean = get(ENABLE_PARQUET) lazy val isParquetInt96WriteEnabled: Boolean = get(ENABLE_PARQUET_INT96_WRITE) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index b9b5741e497..0ec33e2ad3f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -19,8 +19,6 @@ package com.nvidia.spark.rapids import java.io.{File, FileOutputStream} import java.time.ZoneId -import scala.collection.mutable - import ai.rapids.cudf.DType import com.nvidia.spark.rapids.shims.v2.TypeSigUtil @@ -1163,27 +1161,6 @@ object CreateMapCheck extends ExprChecks { val context = exprMeta.context if (context != ProjectExprContext) { meta.willNotWorkOnGpu(s"this is not supported in the $context context") - } else { - // if there are more than two key-value pairs then there is the possibility of duplicate keys - if (meta.childExprs.length > 2) { - // check for duplicate keys if the keys are literal values - val keyExprs = meta.childExprs.indices.filter(_ % 2 == 0).map(meta.childExprs) - val litKeys = keyExprs.map(e => GpuOverrides.extractLit(e.wrapped.asInstanceOf[Expression])) - if (litKeys.forall(_.isDefined)) { - val keys = litKeys.map(_.get.value) - val uniqueKeys = new mutable.HashSet[Any]() - for (key <- keys) { - if (!uniqueKeys.add(key)) { - meta.willNotWorkOnGpu("CreateMap with duplicate literal keys is not supported") - } - } - } else if (!meta.conf.isCreateMapEnabled) { - meta.willNotWorkOnGpu("CreateMap is not enabled by default when there are " + - "multiple key-value pairs and where the keys are not literal values because handling " + - "of duplicate keys is not compatible with Spark. " + - s"Set ${RapidsConf.ENABLE_CREATE_MAP}=true to enable it anyway.") - } - } } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala index 3062aeccf51..61ac28cb9b1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.rapids import ai.rapids.cudf.{ColumnVector, DType} -import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuExpressionsUtils} +import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuExpressionsUtils, GpuMapUtils} import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq import com.nvidia.spark.rapids.shims.v2.ShimExpression @@ -80,7 +80,10 @@ case class GpuCreateArray(children: Seq[Expression], useStringTypeWhenEmpty: Boo } } -case class GpuCreateMap(children: Seq[Expression], useStringTypeWhenEmpty: Boolean) +case class GpuCreateMap( + children: Seq[Expression], + useStringTypeWhenEmpty: Boolean, + isExceptionDedupePolicy: Boolean) extends GpuExpression with ShimExpression { private val valueIndices: Seq[Int] = children.indices.filter(_ % 2 != 0) @@ -106,7 +109,22 @@ case class GpuCreateMap(children: Seq[Expression], useStringTypeWhenEmpty: Boole val structs = Range(0, columns.length, 2) .safeMap(i => ColumnVector.makeStruct(columns(i), columns(i + 1))) withResource(structs) { _ => - GpuColumnVector.from(ColumnVector.makeList(numRows, DType.STRUCT, structs: _*), dataType) + withResource(ColumnVector.makeList(numRows, DType.STRUCT, structs: _*)) { listOfStruct => + withResource(listOfStruct.dropListDuplicatesWithKeysValues()) { deduped => + if (isExceptionDedupePolicy) { + // compare child data row count before and after + // removing duplicates to determine if there were duplicates + withResource(deduped.getChildColumnView(0)) { a => + withResource(listOfStruct.getChildColumnView(0)) { b => + if (a.getRowCount != b.getRowCount) { + throw GpuMapUtils.duplicateMapKeyFoundError + } + } + } + } + GpuColumnVector.from(deduped.incRefCount(), dataType) + } + } } } } @@ -128,7 +146,9 @@ case class GpuCreateMap(children: Seq[Expression], useStringTypeWhenEmpty: Boole object GpuCreateMap { def apply(children: Seq[Expression]): GpuCreateMap = { new GpuCreateMap(children, - SQLConf.get.getConf(SQLConf.LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE)) + SQLConf.get.getConf(SQLConf.LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE), + SQLConf.get.getConf(SQLConf.MAP_KEY_DEDUP_POLICY) == + SQLConf.MapKeyDedupPolicy.EXCEPTION.toString) } } From f4ad8356cded32b1cc615b96ebf7d7bc25490514 Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Thu, 2 Dec 2021 09:23:06 +0800 Subject: [PATCH 06/52] GpuCast from ArrayType to StringType [databricks] (#4221) * Implement casting from array to string for GpuCast Signed-off-by: remzi <13716567376yh@gmail.com> * add integration tests in array_test.py * support legacy cast to string --- docs/supported_ops.md | 4 +- .../src/main/python/array_test.py | 60 ++++++++ .../com/nvidia/spark/rapids/GpuCast.scala | 130 ++++++++++++++++++ .../com/nvidia/spark/rapids/TypeChecks.scala | 2 +- 4 files changed, 193 insertions(+), 3 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 360e1e3dda7..dbabad0d3c9 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -16585,7 +16585,7 @@ and the accelerator produces the same result. -NS +S @@ -16989,7 +16989,7 @@ and the accelerator produces the same result. -NS +S diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index 9362c2181da..762a14fa776 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -249,3 +249,63 @@ def test_sql_array_scalars(query): assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.sql('SELECT {}'.format(query)), conf={'spark.sql.legacy.allowNegativeScaleOfDecimal': 'true'}) + + + +basic_gens_for_cast_to_string = [byte_gen, short_gen, int_gen, long_gen, string_gen, boolean_gen, date_gen, null_gen] + decimal_gens_no_neg +# casting these types to string are not exact match, marked as xfail when testing +not_matched_gens_for_cast_to_string = [float_gen, double_gen, timestamp_gen, decimal_gen_neg_scale] +# casting these types to string are not supported, marked as xfail when testing +not_support_gens_for_cast_to_string = decimal_128_gens + +single_level_array_gens_for_cast_to_string = [ArrayGen(sub_gen) for sub_gen in basic_gens_for_cast_to_string] +nested_array_gens_for_cast_to_string = [ + ArrayGen(ArrayGen(short_gen, max_length=10), max_length=10), + ArrayGen(ArrayGen(string_gen, max_length=10), max_length=10), + ArrayGen(ArrayGen(null_gen, max_length=10), max_length=10), + ArrayGen(StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', date_gen]])) + ] + +all_gens_for_cast_to_string = single_level_array_gens_for_cast_to_string + nested_array_gens_for_cast_to_string + + +@pytest.mark.parametrize('data_gen', all_gens_for_cast_to_string, ids=idfn) +def test_cast_array_to_string(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, data_gen).select( + f.col('a').cast("STRING") + ) + ) + + +@pytest.mark.parametrize('data_gen', all_gens_for_cast_to_string, ids=idfn) +def test_legacy_cast_array_to_string(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, data_gen).select( + f.col('a').cast("STRING") + ), + conf = {'spark.sql.legacy.castComplexTypesToString.enabled': 'true'} + ) + + +@pytest.mark.parametrize('data_gen', [ArrayGen(sub) for sub in not_matched_gens_for_cast_to_string], ids=idfn) +@pytest.mark.xfail(reason='casting these types to string are not exact matchs') +def test_cast_array_with_unmatched_element_to_string(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, data_gen).select( + f.col('a').cast("STRING") + ), + conf = { + "spark.sql.legacy.allowNegativeScaleOfDecimal": "true", + "spark.rapids.sql.castFloatToString.enabled" : "true"} + ) + + +@pytest.mark.parametrize('data_gen', [ArrayGen(sub) for sub in not_support_gens_for_cast_to_string], ids=idfn) +@pytest.mark.xfail(reason='casting these types to string are not supported') +def test_cast_array_with_unsupported_element_to_string(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, data_gen).select( + f.col('a').cast("STRING") + ) +) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index 2110b9e28f8..e2792a5c59e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -530,6 +530,11 @@ object GpuCast extends Arm { } } + case (ArrayType(elementType, _), StringType) => + castArrayToString( + input, elementType, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled + ) + case (from: StructType, to: StructType) => castStructToStruct(from, to, input, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) @@ -636,6 +641,131 @@ object GpuCast extends Arm { } } + + /** + * A 8 steps solution for casting array to string.

+ * Array is represented as `list column` in cudf, which has child column and offset column.

+ * When `legacyCastToString = true`, given an input with 3 rows: + * `[ [1, 2, null, 3], [], null]`

+ * Step 1: cast all not-null elements in array to string type: + * `[ ["1", "2", null, "3"], [], null]`

+ * Step 2: add space char in the front of all not-null elements: + * `[ [" 1", " 2", null, " 3"], [], null]`

+ * step 3: cast `null` elements to their string representation : + * `[ [" 1", " 2", "", " 3"], [], null]`(here we use "" to represent null)

+ * step 4: concatenate list elements, seperated by `","`: + * `[" 1, 2,, 3", null, null]`

+ * step 5: remove the first char, if it is an `' '`: + * `["1, 2,, 3", null, null]`

+ * step 6: replace nulls with empty string: + * `["1, 2,, 3", "", ""]`

+ * step 7: add brackets: + * `["[1, 2,, 3]", "[]", "[]"]`

+ * step 8: add `null` masks using original input: + * `["[1, 2,, 3]", "[]", null]`

+ * + * when `legacyCastToString = false`, step 2, 5 are skipped + */ + private def castArrayToString(input: ColumnView, + elementType: DataType, + ansiMode: Boolean, + legacyCastToString: Boolean, + stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + + val (leftStr, rightStr) = ("[", "]") + val emptyStr = "" + val spaceStr = " " + val nullStr = if (legacyCastToString) emptyStr else "null" + val sepStr = "," + (if (legacyCastToString) emptyStr else spaceStr) + val numRows = input.getRowCount.toInt + + withResource( + Seq(leftStr, rightStr, emptyStr, spaceStr, nullStr, sepStr).safeMap(Scalar.fromString) + ){ case Seq(left, right, empty, space, nullRep, sep) => + + /* -------------------------------- helper functions -----------------------*/ + + /** + * cast all not-null elements in a child column to string type

+ * add `' '` to all elements when `legacyCastToString = true` + * @param child child column of an array column + * @return a string type child column + */ + def castChildToStr(child: ColumnView): ColumnView = { + withResource( + doCast(child, elementType, StringType, ansiMode, + legacyCastToString, stringToDateAnsiModeEnabled) + ) { strChildWithNull => + withResource(strChildWithNull.replaceNulls(nullRep)) { strChild => + if (legacyCastToString) {// add a space string to each non-null element + withResource(ColumnVector.fromScalar(space, child.getRowCount.toInt)) { spaceVec => + withResource( + ColumnVector.stringConcatenate(Array(spaceVec, strChild)) + ) { addSpace => + withResource(child.isNotNull) {_.ifElse(addSpace, strChild)} + } + } + } + else { strChild.incRefCount } + } + } + } + + /** + * If the first char of a string is ' ', remove it (only for legacyCastToString = true) + * @param strVec a string type column vector + */ + def removeFirstSpace(strVec: ColumnVector): ColumnVector = { + if (legacyCastToString){ + withResource(strVec.substring(0,1)) { fstChar => + withResource(strVec.substring(1)) { remain => + withResource(fstChar.equalTo(space)) {_.ifElse(remain, strVec)} + } + } + } + else {strVec.incRefCount} + } + + /** + * Add brackets to each string. Ex: ["1, 2, 3", "4, 5"] => ["[1, 2, 3]", "[4, 5]"] + * @param strVec a string vector + */ + def addBrackets(strVec: ColumnVector): ColumnVector = { + withResource( + Seq(left, right).safeMap(s => ColumnVector.fromScalar(s, numRows)) + ) { case Seq(leftColumn, rightColumn) => + ColumnVector.stringConcatenate(empty, nullRep, Array(leftColumn, strVec, rightColumn)) + } + } + /* -------------------------------- helper functions -----------------------*/ + + + // cast child column to string type + withResource(input.getChildColumnView(0)) { childView => + withResource(castChildToStr(childView)){ stringChild => + withResource(input.replaceListChild(stringChild)) {strArr => + // concatenate each row. cast from list column to string column + withResource(ColumnVector.fromScalar(sep, numRows)){ sepCol => + withResource( + strArr.stringConcatenateListElements(sepCol) + ) { strColContainsNull => + withResource(strColContainsNull.replaceNulls(empty)){strCol => + withResource(removeFirstSpace(strCol)){withoutBrackets => + withResource(addBrackets(withoutBrackets))( + _.mergeAndSetValidity(BinaryOp.BITWISE_AND, input) + ) + } + } + } + } + } + } + } + } + } + + + private def castStructToString( input: ColumnView, inputSchema: Array[StructField], diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 0ec33e2ad3f..e04d442d448 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -1270,7 +1270,7 @@ class CastChecks extends ExprChecks { val calendarChecks: TypeSig = none val sparkCalendarSig: TypeSig = CALENDAR + STRING - val arrayChecks: TypeSig = ARRAY.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + + val arrayChecks: TypeSig = STRING + ARRAY.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + ARRAY + BINARY + STRUCT + MAP) + psNote(TypeEnum.ARRAY, "The array's child type must also support being cast to " + "the desired child type") From 4bbdc28b6f5e991d7e26eaa8229496865b5cdee2 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Wed, 1 Dec 2021 20:40:52 -0600 Subject: [PATCH 07/52] Support row-based Hive UDFs [databricks] (#4224) * Support row-based Hive UDFs Signed-off-by: Firestarman --- aggregator/pom.xml | 16 ++ docs/configs.md | 4 +- docs/supported_ops.md | 4 +- .../src/main/python/row-based_udf_test.py | 43 +++++ sql-plugin/pom.xml | 6 + .../sql/hive/rapids/GpuHiveOverrides.scala | 89 ++++++---- .../sql/hive/rapids/rowBasedHiveUDFs.scala | 155 ++++++++++++++++++ .../rapids/udf/hive/EmptyHiveGenericUDF.java | 57 +++++++ .../rapids/udf/hive/EmptyHiveSimpleUDF.java | 26 +++ 9 files changed, 365 insertions(+), 35 deletions(-) create mode 100644 integration_tests/src/main/python/row-based_udf_test.py create mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala create mode 100644 udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveGenericUDF.java create mode 100644 udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java diff --git a/aggregator/pom.xml b/aggregator/pom.xml index 2ced15b918d..9a429628975 100644 --- a/aggregator/pom.xml +++ b/aggregator/pom.xml @@ -100,9 +100,25 @@ org.apache.hadoop.hive. ${rapids.shade.package}.hadoop.hive. + org.apache.hadoop.hive.conf.HiveConf + org.apache.hadoop.hive.ql.exec.FunctionRegistry org.apache.hadoop.hive.ql.exec.UDF + org.apache.hadoop.hive.ql.exec.UDFMethodResolver + org.apache.hadoop.hive.ql.udf.UDFType org.apache.hadoop.hive.ql.udf.generic.GenericUDF + org.apache.hadoop.hive.ql.udf.generic.GenericUDF$DeferredObject + org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils$ConversionHelper + org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector + org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory + org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory$ObjectInspectorOptions + org.apache.hadoop.hive.serde2.objectinspector.StructField + org.apache.hadoop.hive.serde2.typeinfo.TypeInfo diff --git a/docs/configs.md b/docs/configs.md index 435e08d2161..06614aa4bb4 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -331,8 +331,8 @@ Name | SQL Function(s) | Description | Default Value | Notes spark.rapids.sql.expression.VarianceSamp|`var_samp`, `variance`|Aggregation computing sample variance|true|None| spark.rapids.sql.expression.NormalizeNaNAndZero| |Normalize NaN and zero|true|None| spark.rapids.sql.expression.ScalarSubquery| |Subquery that will return only one row and one column|true|None| -spark.rapids.sql.expression.HiveGenericUDF| |Hive Generic UDF, support requires the UDF to implement a RAPIDS accelerated interface|true|None| -spark.rapids.sql.expression.HiveSimpleUDF| |Hive UDF, support requires the UDF to implement a RAPIDS accelerated interface|true|None| +spark.rapids.sql.expression.HiveGenericUDF| |Hive Generic UDF, the UDF can choose to implement a RAPIDS accelerated interface to get better performance|true|None| +spark.rapids.sql.expression.HiveSimpleUDF| |Hive UDF, the UDF can choose to implement a RAPIDS accelerated interface to get better performance|true|None| ### Execution diff --git a/docs/supported_ops.md b/docs/supported_ops.md index dbabad0d3c9..1aac3b00cc2 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -16149,7 +16149,7 @@ are limited. HiveGenericUDF -Hive Generic UDF, support requires the UDF to implement a RAPIDS accelerated interface +Hive Generic UDF, the UDF can choose to implement a RAPIDS accelerated interface to get better performance None project param @@ -16196,7 +16196,7 @@ are limited. HiveSimpleUDF -Hive UDF, support requires the UDF to implement a RAPIDS accelerated interface +Hive UDF, the UDF can choose to implement a RAPIDS accelerated interface to get better performance None project param diff --git a/integration_tests/src/main/python/row-based_udf_test.py b/integration_tests/src/main/python/row-based_udf_test.py new file mode 100644 index 00000000000..45ae6704746 --- /dev/null +++ b/integration_tests/src/main/python/row-based_udf_test.py @@ -0,0 +1,43 @@ +# Copyright (c) 2021, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import pytest + +from asserts import assert_gpu_and_cpu_are_equal_sql +from data_gen import * +from spark_session import with_spark_session +from rapids_udf_test import skip_if_no_hive, load_hive_udf_or_skip_test + +def test_hive_empty_simple_udf(): + with_spark_session(skip_if_no_hive) + data_gens = [["i", int_gen], ["s", string_gen]] + def evalfn(spark): + load_hive_udf_or_skip_test(spark, "emptysimple", "com.nvidia.spark.rapids.udf.hive.EmptyHiveSimpleUDF") + return gen_df(spark, data_gens) + assert_gpu_and_cpu_are_equal_sql( + evalfn, + "hive_simple_udf_test_table", + "SELECT i, emptysimple(s) FROM hive_simple_udf_test_table", + conf={'spark.rapids.sql.rowBasedUDF.enabled': 'true'}) + +def test_hive_empty_generic_udf(): + with_spark_session(skip_if_no_hive) + def evalfn(spark): + load_hive_udf_or_skip_test(spark, "emptygeneric", "com.nvidia.spark.rapids.udf.hive.EmptyHiveGenericUDF") + return gen_df(spark, [["s", string_gen]]) + assert_gpu_and_cpu_are_equal_sql( + evalfn, + "hive_generic_udf_test_table", + "SELECT emptygeneric(s) FROM hive_generic_udf_test_table", + conf={'spark.rapids.sql.rowBasedUDF.enabled': 'true'}) diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 9111fb89927..67f88916989 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -154,6 +154,12 @@ ${spark.version} provided + + org.apache.hive + hive-serde + ${spark.version} + provided + org.apache.spark spark-hive_${scala.binary.version} diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala index 6384461e0d1..770fcb028fe 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.rapids import com.nvidia.spark.RapidsUDF -import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, RepeatingParamCheck, ShimLoader, TypeSig} +import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, RapidsConf, RepeatingParamCheck, ShimLoader, TypeSig} import com.nvidia.spark.rapids.GpuUserDefinedFunction.udfTypeSig import org.apache.spark.sql.catalyst.expressions.Expression @@ -45,59 +45,86 @@ object GpuHiveOverrides { Seq( GpuOverrides.expr[HiveSimpleUDF]( - "Hive UDF, support requires the UDF to implement a RAPIDS accelerated interface", + "Hive UDF, the UDF can choose to implement a RAPIDS accelerated interface to" + + " get better performance", ExprChecks.projectOnly( udfTypeSig, TypeSig.all, repeatingParamCheck = Some(RepeatingParamCheck("param", udfTypeSig, TypeSig.all))), (a, conf, p, r) => new ExprMeta[HiveSimpleUDF](a, conf, p, r) { + private val opRapidsFunc = a.function match { + case rapidsUDF: RapidsUDF => Some(rapidsUDF) + case _ => None + } + override def tagExprForGpu(): Unit = { - a.function match { - case _: RapidsUDF => - case _ => - willNotWorkOnGpu(s"Hive UDF ${a.name} implemented by " + - s"${a.funcWrapper.functionClassName} does not provide a GPU implementation") + if (opRapidsFunc.isEmpty && !conf.isCpuBasedUDFEnabled) { + willNotWorkOnGpu(s"Hive SimpleUDF ${a.name} implemented by " + + s"${a.funcWrapper.functionClassName} does not provide a GPU implementation " + + s"and CPU-based UDFs are not enabled by `${RapidsConf.ENABLE_CPU_BASED_UDF.key}`") } } override def convertToGpu(): GpuExpression = { - // To avoid adding a Hive dependency just to check if the UDF function is deterministic, - // we use the original HiveSimpleUDF `deterministic` method as a proxy. - GpuHiveSimpleUDF( - a.name, - a.funcWrapper, - childExprs.map(_.convertToGpu()), - a.dataType, - a.deterministic) + opRapidsFunc.map { _ => + // We use the original HiveGenericUDF `deterministic` method as a proxy + // for simplicity. + GpuHiveSimpleUDF( + a.name, + a.funcWrapper, + childExprs.map(_.convertToGpu()), + a.dataType, + a.deterministic) + }.getOrElse { + // This `require` is just for double check. + require(conf.isCpuBasedUDFEnabled) + GpuRowBasedHiveSimpleUDF( + a.name, + a.funcWrapper, + childExprs.map(_.convertToGpu())) + } } }), GpuOverrides.expr[HiveGenericUDF]( - "Hive Generic UDF, support requires the UDF to implement a " + - "RAPIDS accelerated interface", + "Hive Generic UDF, the UDF can choose to implement a RAPIDS accelerated interface to" + + " get better performance", ExprChecks.projectOnly( udfTypeSig, TypeSig.all, repeatingParamCheck = Some(RepeatingParamCheck("param", udfTypeSig, TypeSig.all))), (a, conf, p, r) => new ExprMeta[HiveGenericUDF](a, conf, p, r) { + private val opRapidsFunc = a.function match { + case rapidsUDF: RapidsUDF => Some(rapidsUDF) + case _ => None + } + override def tagExprForGpu(): Unit = { - a.function match { - case _: RapidsUDF => - case _ => - willNotWorkOnGpu(s"Hive GenericUDF ${a.name} implemented by " + - s"${a.funcWrapper.functionClassName} does not provide a GPU implementation") + if (opRapidsFunc.isEmpty && !conf.isCpuBasedUDFEnabled) { + willNotWorkOnGpu(s"Hive GenericUDF ${a.name} implemented by " + + s"${a.funcWrapper.functionClassName} does not provide a GPU implementation " + + s"and CPU-based UDFs are not enabled by `${RapidsConf.ENABLE_CPU_BASED_UDF.key}`") } } override def convertToGpu(): GpuExpression = { - // To avoid adding a Hive dependency just to check if the UDF function is deterministic, - // we use the original HiveGenericUDF `deterministic` method as a proxy. - GpuHiveGenericUDF( - a.name, - a.funcWrapper, - childExprs.map(_.convertToGpu()), - a.dataType, - a.deterministic, - a.foldable) + opRapidsFunc.map { _ => + // We use the original HiveGenericUDF `deterministic` method as a proxy + // for simplicity. + GpuHiveGenericUDF( + a.name, + a.funcWrapper, + childExprs.map(_.convertToGpu()), + a.dataType, + a.deterministic, + a.foldable) + }.getOrElse { + // This `require` is just for double check. + require(conf.isCpuBasedUDFEnabled) + GpuRowBasedHiveGenericUDF( + a.name, + a.funcWrapper, + childExprs.map(_.convertToGpu())) + } } }) ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala new file mode 100644 index 00000000000..cc0ddfcbd86 --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala @@ -0,0 +1,155 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.rapids + +import scala.collection.JavaConverters._ + +import com.nvidia.spark.rapids.GpuRowBasedUserDefinedFunction +import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, UDF} +import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper +import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Expression, SpecializedGetters} +import org.apache.spark.sql.hive.{DeferredObjectAdapter, HiveInspectors} +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper +import org.apache.spark.sql.types.DataType + +/** Common implementation across row-based Hive UDFs */ +trait GpuRowBasedHiveUDFBase extends GpuRowBasedUserDefinedFunction with HiveInspectors { + val funcWrapper: HiveFunctionWrapper + + @transient + val function: AnyRef + + override val udfDeterministic: Boolean = { + val udfType = function.getClass.getAnnotation(classOf[HiveUDFType]) + udfType != null && udfType.deterministic() && !udfType.stateful() + } + + override final val checkNull: Boolean = false + + override def nullable: Boolean = true + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } + + override def prettyName: String = name + + @transient + protected lazy val childRowAccessors: Array[SpecializedGetters => Any] = + children.zipWithIndex.map { case (child, i) => + val accessor = InternalRow.getAccessor(child.dataType, child.nullable) + row: SpecializedGetters => accessor(row, i) + }.toArray + + @transient + protected lazy val argumentInspectors = children.map(toInspector) +} + +/** Row-based version of Spark's `HiveSimpleUDF` running in a GPU operation */ +case class GpuRowBasedHiveSimpleUDF( + name: String, + funcWrapper: HiveFunctionWrapper, + children: Seq[Expression]) extends GpuRowBasedHiveUDFBase { + + @scala.annotation.nowarn("msg=class UDF in package exec is deprecated") + @transient + override lazy val function: UDF = funcWrapper.createFunction[UDF]() + + @transient + private lazy val wrappers = children.map(x => wrapperFor(toInspector(x), x.dataType)).toArray + + @transient + private lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) + + @transient + private lazy val inputDataTypes: Array[DataType] = children.map(_.dataType).toArray + + @transient + private lazy val method = + function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo).asJava) + + // Create parameter converters + @transient + private lazy val conversionHelper = new ConversionHelper(method, argumentInspectors.toArray) + + @transient + private lazy val unwrapper = unwrapperFor( + ObjectInspectorFactory.getReflectionObjectInspector( + method.getGenericReturnType, ObjectInspectorOptions.JAVA)) + + override protected def evaluateRow(childrenRow: InternalRow): Any = { + val inputs = wrap(childRowAccessors.map(_(childrenRow)), wrappers, cached, inputDataTypes) + val ret = FunctionRegistry.invoke( + method, + function, + conversionHelper.convertIfNecessary(inputs : _*): _*) + unwrapper(ret) + } + + override lazy val dataType: DataType = javaTypeToDataType(method.getGenericReturnType) + + override def foldable: Boolean = udfDeterministic && children.forall(_.foldable) + + override def sql: String = s"$name(${children.map(_.sql).mkString(", ")})" +} + +/** Row-based version of Spark's `HiveGenericUDF` running in a GPU operation */ +case class GpuRowBasedHiveGenericUDF( + name: String, + funcWrapper: HiveFunctionWrapper, + children: Seq[Expression]) extends GpuRowBasedHiveUDFBase { + + @transient + override lazy val function: GenericUDF = funcWrapper.createFunction[GenericUDF]() + + @transient + private lazy val returnInspector = + function.initializeAndFoldConstants(argumentInspectors.toArray) + + @transient + private lazy val deferredObjects = argumentInspectors.zip(children).map { + case (inspect, child) => new DeferredObjectAdapter(inspect, child.dataType) + }.toArray + + @transient + private lazy val unwrapper = unwrapperFor(returnInspector) + + override protected def evaluateRow(childrenRow: InternalRow): Any = { + returnInspector // Make sure initialized. + + var i = 0 + val length = children.length + while (i < length) { + val idx = i + deferredObjects(i).set(() => childRowAccessors(idx)(childrenRow)) + i += 1 + } + unwrapper(function.evaluate(deferredObjects.asInstanceOf[Array[DeferredObject]])) + } + + override lazy val dataType: DataType = inspectorToDataType(returnInspector) + + override def foldable: Boolean = + udfDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector] +} diff --git a/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveGenericUDF.java b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveGenericUDF.java new file mode 100644 index 00000000000..b026936eb7e --- /dev/null +++ b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveGenericUDF.java @@ -0,0 +1,57 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.udf.hive; + +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.Text; + +/** An empty Hive GenericUDF returning the input directly for row-based UDF test only */ +public class EmptyHiveGenericUDF extends GenericUDF { + private transient PrimitiveObjectInspectorConverter.TextConverter converter; + private final Text textResult = new Text(); + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + if (arguments.length != 1) { + throw new UDFArgumentException("One argument is supported, but found: " + arguments.length); + } + if (!(arguments[0] instanceof PrimitiveObjectInspector)) { + throw new UDFArgumentException("Unsupported argument type: " + arguments[0].getTypeName()); + } + PrimitiveObjectInspector poi = (PrimitiveObjectInspector) arguments[0]; + converter = new PrimitiveObjectInspectorConverter.TextConverter(poi); + return PrimitiveObjectInspectorFactory.writableStringObjectInspector; + } + + @Override + public Object evaluate(DeferredObject[] deferredObjects) throws HiveException { + Text text = converter.convert(deferredObjects[0].get()); + textResult.set(text == null ? "" : text.toString()); + return textResult; + } + + @Override + public String getDisplayString(String[] strings) { + return getStandardDisplayString("empty", strings); + } +} diff --git a/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java new file mode 100644 index 00000000000..43c494e2ecf --- /dev/null +++ b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java @@ -0,0 +1,26 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.udf.hive; + +import org.apache.hadoop.hive.ql.exec.UDF; + +/** An empty Hive simple UDF returning the input directly for row-based UDF test only. */ +public class EmptyHiveSimpleUDF extends UDF { + public String evaluate(String in) { + return in; + } +} From 1d961d3c9de564e524870c1dd8ecb784f7b0fc13 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 Dec 2021 21:08:34 -0700 Subject: [PATCH 08/52] fix bug in parsing regex character classes that start with `^` and contain an unescaped `]` (#4241) Signed-off-by: Andy Grove --- .../scala/com/nvidia/spark/rapids/RegexParser.scala | 4 +++- .../spark/rapids/RegularExpressionParserSuite.scala | 13 +++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 86ba30d9c83..a2d4d347fb1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -150,9 +150,11 @@ class RegexParser(pattern: String) { case '[' => // treat as a literal character and add to the character class characterClass.append(ch) - case ']' if pos > start + 1 => + case ']' if (!characterClass.negated && pos > start + 1) || + (characterClass.negated && pos > start + 2) => // "[]" is not a valid character class // "[]a]" is a valid character class containing the characters "]" and "a" + // "[^]a]" is a valid negated character class containing the characters "]" and "a" characterClassComplete = true case '^' if pos == start + 1 => // Negates the character class, causing it to match a single character not listed in diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala index aac297eea0a..4836c8f15cb 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala @@ -68,12 +68,25 @@ class RegularExpressionParserSuite extends FunSuite { RegexCharacterRange('A', 'Z')))))) } + test("character class complex example") { + assert(parse("[^]+d]+") === RegexSequence(ListBuffer( + RegexRepetition( + RegexCharacterClass(negated = true, + ListBuffer(RegexChar(']'), RegexChar('+'), RegexChar('d'))), + SimpleQuantifier('+'))))) + } + test("character classes containing ']'") { // "[]a]" is a valid character class containing ']' and 'a' assert(parse("[]a]") === RegexSequence(ListBuffer( RegexCharacterClass(negated = false, ListBuffer(RegexChar(']'), RegexChar('a')))))) + // "[^]a]" is a valid negated character class containing ']' and 'a' + assert(parse("[^]a]") === + RegexSequence(ListBuffer( + RegexCharacterClass(negated = true, + ListBuffer(RegexChar(']'), RegexChar('a')))))) // "[a]]" is a valid character class "[a]" followed by character ']' assert(parse("[a]]") === RegexSequence(ListBuffer( From 90d846b19014ffcb9e52ac01c46398a7c1728aa9 Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Thu, 2 Dec 2021 14:24:53 +0800 Subject: [PATCH 09/52] Update the release script [skip ci] (#4250) Argument for the spark version of the final build and aggregation Signed-off-by: Tim Liu --- jenkins/deploy.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/jenkins/deploy.sh b/jenkins/deploy.sh index 6ea9d24fae0..e218c736964 100755 --- a/jenkins/deploy.sh +++ b/jenkins/deploy.sh @@ -28,6 +28,7 @@ # SERVER_ID: The repository id for this deployment. # SERVER_URL: The url where to deploy artifacts. # GPG_PASSPHRASE: The passphrase used to sign files, only required when is true. +# FINAL_AGG_VERSION_TOBUILD: The spark version of the final build and aggregation. ### set -ex @@ -55,14 +56,14 @@ FPATH="$DIST_PL/target/$ART_ID-$ART_VER" echo "Plan to deploy ${FPATH}.jar to $SERVER_URL (ID:$SERVER_ID)" - +FINAL_AGG_VERSION_TOBUILD=${FINAL_AGG_VERSION_TOBUILD:-'301'} ###### Choose the deploy command ###### if [ "$SIGN_FILE" == true ]; then # No javadoc and sources jar is generated for shade artifact only. Use 'sql-plugin' instead SQL_ART_ID=`mvn help:evaluate -q -pl $SQL_PL -Dexpression=project.artifactId -DforceStdout` SQL_ART_VER=`mvn help:evaluate -q -pl $SQL_PL -Dexpression=project.version -DforceStdout` - JS_FPATH="${SQL_PL}/target/${SQL_ART_ID}-${SQL_ART_VER}" + JS_FPATH="${SQL_PL}/target/spark${FINAL_AGG_VERSION_TOBUILD}/${SQL_ART_ID}-${SQL_ART_VER}" SRC_DOC_JARS="-Dsources=${JS_FPATH}-sources.jar -Djavadoc=${JS_FPATH}-javadoc.jar" DEPLOY_CMD="mvn -B gpg:sign-and-deploy-file -s jenkins/settings.xml -Dgpg.passphrase=$GPG_PASSPHRASE" else @@ -82,15 +83,15 @@ $DEPLOY_CMD -Durl=$SERVER_URL -DrepositoryId=$SERVER_ID \ # Distribution jar is a shaded artifact so use the reduced dependency pom. $DEPLOY_CMD -Durl=$SERVER_URL -DrepositoryId=$SERVER_ID \ $SRC_DOC_JARS \ - -Dfile=$FPATH.jar -DgroupId=com.nvidia -DartifactId=$ART_ID -Dversion=$ART_VER -DpomFile=./dist/target/dependency-reduced-pom.xml + -Dfile=$FPATH.jar -DgroupId=com.nvidia -DartifactId=$ART_ID -Dversion=$ART_VER -DpomFile=./dist/pom.xml ###### Deploy integration tests jar(s) ###### TESTS_ART_ID=`mvn help:evaluate -q -pl $TESTS_PL -Dexpression=project.artifactId -DforceStdout` TESTS_ART_VER=`mvn help:evaluate -q -pl $TESTS_PL -Dexpression=project.version -DforceStdout` TESTS_DOC_JARS="-Dsources=deployjars/$TESTS_ART_ID-$TESTS_ART_VER-sources.jar -Djavadoc=deployjars/$TESTS_ART_ID-$TESTS_ART_VER-javadoc.jar" -# Deploy default integration tests jar (spark301) +# Copy the final aggregation jar as the default integration-tests jar TESTS_FPATH="deployjars/$TESTS_ART_ID-$TESTS_ART_VER" -cp $TESTS_FPATH-spark301.jar $TESTS_FPATH.jar +cp $TESTS_FPATH-spark${FINAL_AGG_VERSION_TOBUILD}.jar $TESTS_FPATH.jar $DEPLOY_CMD -Durl=$SERVER_URL -DrepositoryId=$SERVER_ID \ $TESTS_DOC_JARS \ -Dfile=$TESTS_FPATH.jar -DpomFile=${TESTS_PL}/pom.xml From bad9fee828aad01cb61b5bbc7eac42d4f9d18c09 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 2 Dec 2021 08:54:57 -0600 Subject: [PATCH 10/52] Implement getShuffleRDD and fixup mismatched output types on shuffle reuse [databricks] (#4257) * Implement getShuffleRDD and fixup mismatched output types on shuffle reuse Signed-off-by: Jason Lowe * Fix Databricks 9.1 build --- .../src/main/python/repart_test.py | 11 +++++++ .../spark/rapids/shims/v2/AQEUtils.scala | 28 ++++++++++++++++ .../spark/rapids/shims/v2/AQEUtils.scala | 28 ++++++++++++++++ .../shims/v2/GpuShuffleExchangeExec.scala | 4 +-- .../shims/v2/GpuShuffleExchangeExec.scala | 4 +-- .../shims/v2/GpuShuffleExchangeExec.scala | 4 +-- .../spark/rapids/shims/v2/AQEUtils.scala | 30 +++++++++++++++++ .../shims/v2/GpuShuffleExchangeExec.scala | 4 +-- .../nvidia/spark/rapids/GpuOverrides.scala | 32 +++++++++++++++++-- 9 files changed, 135 insertions(+), 10 deletions(-) create mode 100644 sql-plugin/src/main/301+-nondb/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala create mode 100644 sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala create mode 100644 sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala diff --git a/integration_tests/src/main/python/repart_test.py b/integration_tests/src/main/python/repart_test.py index 4c6116f5f48..8b8df12c2cb 100644 --- a/integration_tests/src/main/python/repart_test.py +++ b/integration_tests/src/main/python/repart_test.py @@ -267,3 +267,14 @@ def test_hash_repartition_exact(gen, num_parts): .withColumn('hashed', f.hash(*part_on))\ .selectExpr('*', 'pmod(hashed, {})'.format(num_parts)), conf = allow_negative_scale_of_decimal_conf) + +# Test a query that should cause Spark to leverage getShuffleRDD +@ignore_order(local=True) +def test_union_with_filter(): + def doit(spark): + dfa = spark.range(1, 100).withColumn("id2", f.col("id")) + dfb = dfa.groupBy("id").agg(f.size(f.collect_set("id2")).alias("idc")) + dfc = dfb.filter(f.col("idc") == 1).select("id") + return dfc.union(dfc) + conf = { "spark.sql.adaptive.enabled": "true" } + assert_gpu_and_cpu_are_equal_collect(doit, conf) diff --git a/sql-plugin/src/main/301+-nondb/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala b/sql-plugin/src/main/301+-nondb/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala new file mode 100644 index 00000000000..2adb6d96c10 --- /dev/null +++ b/sql-plugin/src/main/301+-nondb/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala @@ -0,0 +1,28 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.v2 + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.adaptive.{QueryStageExec, ShuffleQueryStageExec} + +/** Utility methods for manipulating Catalyst classes involved in Adaptive Query Execution */ +object AQEUtils { + /** Return a new QueryStageExec reuse instance with updated output attributes */ + def newReuseInstance(sqse: ShuffleQueryStageExec, newOutput: Seq[Attribute]): QueryStageExec = { + sqse.newReuseInstance(sqse.id, newOutput) + } +} diff --git a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala new file mode 100644 index 00000000000..2adb6d96c10 --- /dev/null +++ b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala @@ -0,0 +1,28 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.v2 + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.adaptive.{QueryStageExec, ShuffleQueryStageExec} + +/** Utility methods for manipulating Catalyst classes involved in Adaptive Query Execution */ +object AQEUtils { + /** Return a new QueryStageExec reuse instance with updated output attributes */ + def newReuseInstance(sqse: ShuffleQueryStageExec, newOutput: Seq[Attribute]): QueryStageExec = { + sqse.newReuseInstance(sqse.id, newOutput) + } +} diff --git a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala index 0a8518ce8f0..1555df344bc 100644 --- a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan} import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike -import org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBaseWithMetrics +import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBaseWithMetrics, ShuffledBatchRDD} case class GpuShuffleExchangeExec( gpuOutputPartitioning: GpuPartitioning, @@ -43,7 +43,7 @@ case class GpuShuffleExchangeExec( override def getShuffleRDD( partitionSpecs: Array[ShufflePartitionSpec], partitionSizes: Option[Array[Long]]): RDD[_] = { - throw new UnsupportedOperationException + new ShuffledBatchRDD(shuffleDependencyColumnar, metrics ++ readMetrics, partitionSpecs) } override def runtimeStatistics: Statistics = { diff --git a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala index fd686fc1693..ed7d5a18557 100644 --- a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan} import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike -import org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBaseWithMetrics +import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBaseWithMetrics, ShuffledBatchRDD} case class GpuShuffleExchangeExec( gpuOutputPartitioning: GpuPartitioning, @@ -41,7 +41,7 @@ case class GpuShuffleExchangeExec( override def numPartitions: Int = shuffleDependencyColumnar.partitioner.numPartitions override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[_] = { - throw new UnsupportedOperationException + new ShuffledBatchRDD(shuffleDependencyColumnar, metrics ++ readMetrics, partitionSpecs) } override def runtimeStatistics: Statistics = { diff --git a/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala index 003fe861e05..8e325371a72 100644 --- a/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan} import org.apache.spark.sql.execution.exchange.{ShuffleExchangeLike, ShuffleOrigin} -import org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBaseWithMetrics +import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBaseWithMetrics, ShuffledBatchRDD} case class GpuShuffleExchangeExec( gpuOutputPartitioning: GpuPartitioning, @@ -42,7 +42,7 @@ case class GpuShuffleExchangeExec( override def numPartitions: Int = shuffleDependencyColumnar.partitioner.numPartitions override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[_] = { - throw new UnsupportedOperationException + new ShuffledBatchRDD(shuffleDependencyColumnar, metrics ++ readMetrics, partitionSpecs) } override def runtimeStatistics: Statistics = { diff --git a/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala b/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala new file mode 100644 index 00000000000..df2aee9268c --- /dev/null +++ b/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala @@ -0,0 +1,30 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.v2 + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.adaptive.{QueryStageExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec + +/** Utility methods for manipulating Catalyst classes involved in Adaptive Query Execution */ +object AQEUtils { + /** Return a new QueryStageExec reuse instance with updated output attributes */ + def newReuseInstance(sqse: ShuffleQueryStageExec, newOutput: Seq[Attribute]): QueryStageExec = { + val reusedExchange = ReusedExchangeExec(newOutput, sqse.shuffle) + ShuffleQueryStageExec(sqse.id, reusedExchange, sqse.originalPlan) + } +} diff --git a/sql-plugin/src/main/312db/scala/org/apache/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/312db/scala/org/apache/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala index 1ecbf9b133d..81b6131eca2 100644 --- a/sql-plugin/src/main/312db/scala/org/apache/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/312db/scala/org/apache/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan} import org.apache.spark.sql.execution.exchange.{ShuffleExchangeLike, ShuffleOrigin} -import org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase +import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBase, ShuffledBatchRDD} case class GpuShuffleExchangeExec( gpuOutputPartitioning: GpuPartitioning, @@ -54,7 +54,7 @@ case class GpuShuffleExchangeExec( override def getShuffleRDD( partitionSpecs: Array[ShufflePartitionSpec], partitionSizes: Option[Array[Long]]): RDD[_] = { - throw new UnsupportedOperationException + new ShuffledBatchRDD(shuffleDependencyColumnar, metrics ++ readMetrics, partitionSpecs) } // DB SPECIFIC - throw if called since we don't know how its used diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 919ffc775e3..abeb3f1fb19 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -24,7 +24,7 @@ import scala.util.control.NonFatal import ai.rapids.cudf.DType import com.nvidia.spark.rapids.RapidsConf.{SUPPRESS_PLANNING_FAILURE, TEST_CONF} -import com.nvidia.spark.rapids.shims.v2.{GpuSpecifiedWindowFrameMeta, GpuWindowExpressionMeta, OffsetWindowFunctionMeta} +import com.nvidia.spark.rapids.shims.v2.{AQEUtils, GpuSpecifiedWindowFrameMeta, GpuWindowExpressionMeta, OffsetWindowFunctionMeta} import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SparkSession} @@ -548,6 +548,30 @@ object GpuOverrides extends Logging { } } + /** + * Searches the plan for ReusedExchangeExec instances containing a GPU shuffle where the + * output types between the two plan nodes do not match. In such a case the ReusedExchangeExec + * will be updated to match the GPU shuffle output types. + */ + def fixupReusedExchangeExecs(plan: SparkPlan): SparkPlan = { + def outputTypesMatch(a: Seq[Attribute], b: Seq[Attribute]): Boolean = + a.corresponds(b)((x, y) => x.dataType == y.dataType) + plan.transformUp { + case sqse: ShuffleQueryStageExec => + sqse.plan match { + case ReusedExchangeExec(output, gsee: GpuShuffleExchangeExecBase) if ( + !outputTypesMatch(output, gsee.output)) => + val newOutput = sqse.plan.output.zip(gsee.output).map { case (c, g) => + assert(c.isInstanceOf[AttributeReference] && g.isInstanceOf[AttributeReference], + s"Expected AttributeReference but found $c and $g") + AttributeReference(c.name, g.dataType, c.nullable, c.metadata)(c.exprId, c.qualifier) + } + AQEUtils.newReuseInstance(sqse, newOutput) + case _ => sqse + } + } + } + @scala.annotation.tailrec def extractLit(exp: Expression): Option[Literal] = exp match { case l: Literal => Some(l) @@ -3910,7 +3934,11 @@ case class GpuOverrides() extends Rule[SparkPlan] with Logging { val updatedPlan = if (plan.conf.adaptiveExecutionEnabled) { // AQE can cause Spark to inject undesired CPU shuffles into the plan because GPU and CPU // distribution expressions are not semantically equal. - GpuOverrides.removeExtraneousShuffles(plan, conf) + val newPlan = GpuOverrides.removeExtraneousShuffles(plan, conf) + + // AQE can cause ReusedExchangeExec instance to cache the wrong aggregation buffer type + // compared to the desired buffer type from a reused GPU shuffle. + GpuOverrides.fixupReusedExchangeExecs(newPlan) } else { plan } From 8731ee5aa426e02f4babbd78fa023b8e7fc06344 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Dec 2021 08:29:56 -0700 Subject: [PATCH 11/52] fix regression in cast from string to float that caused signed NaN to be valid (#4259) Signed-off-by: Andy Grove --- .../com/nvidia/spark/rapids/GpuCast.scala | 69 ++++++++----------- .../com/nvidia/spark/rapids/CastOpSuite.scala | 12 ++-- 2 files changed, 35 insertions(+), 46 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index e2792a5c59e..dec844f5f10 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -173,26 +173,30 @@ object GpuCast extends Arm { input: ColumnVector, ansiEnabled: Boolean): ColumnVector = { - // This regex gets applied after the transformation to normalize use of Inf and is - // just strict enough to filter out known edge cases that would result in incorrect - // values. We further filter out invalid values using the cuDF isFloat method. + // This regex is just strict enough to filter out known edge cases that would result + // in incorrect values. We further filter out invalid values using the cuDF isFloat method. val VALID_FLOAT_REGEX = - "^" + // start of line - "[+\\-]?" + // optional + or - at start of string - "(" + + "^" + // start of line + "[Nn][Aa][Nn]" + // NaN + "|" + "(" + + "[+\\-]?" + // optional sign preceding Inf or numeric "(" + - "([0-9]+)|" + // digits, OR - "([0-9]*\\.[0-9]+)|" + // decimal with optional leading and mandatory trailing, OR - "([0-9]+\\.[0-9]*)" + // decimal with mandatory leading and optional trailing + "([Ii][Nn][Ff]" + // Inf, Infinity + "([Ii][Nn][Ii][Tt][Yy])?)" + + "|" + + "(" + + "(" + + "([0-9]+)|" + // digits, OR + "([0-9]*\\.[0-9]+)|" + // decimal with optional leading and mandatory trailing, OR + "([0-9]+\\.[0-9]*)" + // decimal with mandatory leading and optional trailing + ")" + + "([eE][+\\-]?[0-9]+)?" + // exponent + "[fFdD]?" + // floating-point designator + ")" + ")" + - "([eE][+\\-]?[0-9]+)?" + // exponent - "[fFdD]?" + // floating-point designator ")" + - "|Inf" + // Infinity - "|[nN][aA][nN]" + // NaN - ")" + - "$" // end of line + "$" // end of line withResource(input.lstrip()) { stripped => withResource(GpuScalar.from(null, DataTypes.StringType)) { nullString => @@ -203,39 +207,26 @@ object GpuCast extends Arm { _.ifElse(nullString, stripped) } } - // replace all possible versions of "Inf" and "Infinity" with "Inf" - val inf = withResource(withoutWhitespace) { _ => - withoutWhitespace.stringReplaceWithBackrefs( - "(?:[iI][nN][fF])" + "(?:[iI][nN][iI][tT][yY])?", "Inf") - } - // replace "+Inf" with "Inf" because cuDF only supports "Inf" and "-Inf" - val infWithoutPlus = withResource(inf) { _ => - withResource(GpuScalar.from("+Inf", DataTypes.StringType)) { search => - withResource(GpuScalar.from("Inf", DataTypes.StringType)) { replace => - inf.stringReplace(search, replace) - } - } - } // filter out any strings that are not valid floating point numbers according // to the regex pattern - val floatOrNull = withResource(infWithoutPlus) { _ => - withResource(infWithoutPlus.matchesRe(VALID_FLOAT_REGEX)) { isFloat => + val floatOrNull = withResource(withoutWhitespace) { _ => + withResource(withoutWhitespace.matchesRe(VALID_FLOAT_REGEX)) { isFloat => if (ansiEnabled) { withResource(isFloat.all()) { allMatch => // Check that all non-null values are valid floats. if (allMatch.isValid && !allMatch.getBoolean) { throw new NumberFormatException(GpuCast.INVALID_NUMBER_MSG) } - infWithoutPlus.incRefCount() + withoutWhitespace.incRefCount() } } else { - isFloat.ifElse(infWithoutPlus, nullString) + isFloat.ifElse(withoutWhitespace, nullString) } } } // strip floating-point designator 'f' or 'd' but don't strip the 'f' from 'Inf' withResource(floatOrNull) { - _.stringReplaceWithBackrefs("([^n])[fFdD]$", "\\1") + _.stringReplaceWithBackrefs("([^nN])[fFdD]$", "\\1") } } } @@ -961,13 +952,11 @@ object GpuCast extends Arm { input: ColumnVector, ansiEnabled: Boolean, dType: DType): ColumnVector = { - // 1. convert the different infinities to "Inf"/"-Inf" which is the only variation cudf - // understands - // 2. identify the nans - // 3. identify the floats. "nan", "null" and letters are not considered floats - // 4. if ansi is enabled we want to throw an exception if the string is neither float nor nan - // 5. convert everything that's not floats to null - // 6. set the indices where we originally had nans to Float.NaN + // 1. identify the nans + // 2. identify the floats. "null" and letters are not considered floats + // 3. if ansi is enabled we want to throw an exception if the string is neither float nor nan + // 4. convert everything that's not floats to null + // 5. set the indices where we originally had nans to Float.NaN // // NOTE Limitation: "1.7976931348623159E308" and "-1.7976931348623159E308" are not considered // Inf even though Spark does diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala index de3f7acc3d7..938b830a6ec 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala @@ -126,8 +126,7 @@ class CastOpSuite extends GpuExpressionTestSuite { testCastStringTo(DataTypes.FloatType, generateRandomStrings(Some(NUMERIC_CHARS))) } - // https://github.com/NVIDIA/spark-rapids/issues/4246 - ignore("Cast from string to float using hand-picked values") { + test("Cast from string to float using hand-picked values") { testCastStringTo(DataTypes.FloatType, Seq(".", "e", "Infinity", "+Infinity", "-Infinity", "+nAn", "-naN", "Nan", "5f", "1.2f", "\riNf", null)) } @@ -933,10 +932,11 @@ class CastOpSuite extends GpuExpressionTestSuite { test("CAST string to float - sanitize step") { val testPairs = Seq( - ("\tinf", "Inf"), - ("\t+InFinITy", "Inf"), - ("\tInFinITy", "Inf"), - ("\t-InFinITy", "-Inf"), + ("\tinf", "inf"), + ("\riNf", "iNf"), + ("\t+InFinITy", "+InFinITy"), + ("\tInFinITy", "InFinITy"), + ("\t-InFinITy", "-InFinITy"), ("\t61f", "61"), (".8E4f", ".8E4") ) From 08d295a05aaf2ae52d12a0338377580471de7024 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Dec 2021 09:03:24 -0700 Subject: [PATCH 12/52] Add tests for regexp() and regexp_like() [databricks] (#4093) * Add tests for regexp and regexplike (synonyms for RLike) Signed-off-by: Andy Grove * add import --- .../src/main/python/string_test.py | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/integration_tests/src/main/python/string_test.py b/integration_tests/src/main/python/string_test.py index 48032f14b20..0581250ed74 100644 --- a/integration_tests/src/main/python/string_test.py +++ b/integration_tests/src/main/python/string_test.py @@ -20,6 +20,7 @@ from marks import * from pyspark.sql.types import * import pyspark.sql.functions as f +from spark_session import is_before_spark_320 def mk_str_gen(pattern): return StringGen(pattern).with_special_case('').with_special_pattern('.{0,10}') @@ -483,6 +484,28 @@ def test_regexp_replace(): 'regexp_replace(a, "a|b|c", "A")'), conf={'spark.rapids.sql.expression.RegExpReplace': 'true'}) +@pytest.mark.skipif(is_before_spark_320(), reason='regexp is synonym for RLike starting in Spark 3.2.0') +def test_regexp(): + gen = mk_str_gen('[abcd]{1,3}') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp(a, "a{2}")', + 'regexp(a, "a{1,3}")', + 'regexp(a, "a{1,}")', + 'regexp(a, "a[bc]d")'), + conf={'spark.rapids.sql.expression.RLike': 'true'}) + +@pytest.mark.skipif(is_before_spark_320(), reason='regexp_like is synonym for RLike starting in Spark 3.2.0') +def test_regexp_like(): + gen = mk_str_gen('[abcd]{1,3}') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp_like(a, "a{2}")', + 'regexp_like(a, "a{1,3}")', + 'regexp_like(a, "a{1,}")', + 'regexp_like(a, "a[bc]d")'), + conf={'spark.rapids.sql.expression.RLike': 'true'}) + @pytest.mark.skipif(is_databricks_runtime(), reason='Databricks optimizes out regexp_replace call in this case') @allow_non_gpu('ProjectExec', 'RegExpReplace') From 588a9e3fbb21b9730a50a5d5093de7a692854013 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 2 Dec 2021 15:12:40 -0600 Subject: [PATCH 13/52] Document exponent differences when casting floating point to string (#4270) Signed-off-by: Jason Lowe --- docs/compatibility.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index 125256fbe6e..d75a8b423be 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -725,7 +725,8 @@ This configuration setting is ignored when using Spark versions prior to 3.1.0. ### Float to String The GPU will use different precision than Java's toString method when converting floating-point data -types to strings and this can produce results that differ from the default behavior in Spark. +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`. From 1cb94d8b3d03502c5a12e26e5e350fb0d13489b4 Mon Sep 17 00:00:00 2001 From: Peixin Date: Fri, 3 Dec 2021 09:51:01 +0800 Subject: [PATCH 14/52] use mamba for cudf conda install (#4278) Signed-off-by: Peixin Li --- jenkins/Dockerfile-blossom.integration.centos | 9 +++++---- jenkins/Dockerfile-blossom.integration.ubuntu | 9 +++++---- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/jenkins/Dockerfile-blossom.integration.centos b/jenkins/Dockerfile-blossom.integration.centos index 9f5504fc8ad..f521e7e8921 100644 --- a/jenkins/Dockerfile-blossom.integration.centos +++ b/jenkins/Dockerfile-blossom.integration.centos @@ -48,10 +48,11 @@ RUN wget --quiet https://repo.anaconda.com/miniconda/Miniconda3-latest-Linux-x86 ENV PATH="/opt/conda/bin:$MAVEN_HOME/bin:${PATH}" # 'pyarrow' and 'pandas' will be installed as the dependencies of cudf below RUN export CUDA_VER=`echo ${CUDA_VER} | cut -d '.' -f 1,2` && \ - conda install -y -c rapidsai -c rapidsai-nightly -c nvidia -c conda-forge -c defaults cudf=${CUDF_VER} python=3.8 cudatoolkit=${CUDA_VER} && \ - conda install -y spacy && python -m spacy download en_core_web_sm && \ - conda install -y -c anaconda pytest requests && \ - conda install -y -c conda-forge sre_yield && \ + conda install -c conda-forge mamba && \ + mamba install -y -c rapidsai -c rapidsai-nightly -c nvidia -c conda-forge -c defaults cudf=${CUDF_VER} python=3.8 cudatoolkit=${CUDA_VER} && \ + mamba install -y spacy && python -m spacy download en_core_web_sm && \ + mamba install -y -c anaconda pytest requests && \ + mamba install -y -c conda-forge sre_yield && \ conda clean -ay # Set default java as 1.8.0 diff --git a/jenkins/Dockerfile-blossom.integration.ubuntu b/jenkins/Dockerfile-blossom.integration.ubuntu index a5ec0c2b9bc..b338aae58b2 100644 --- a/jenkins/Dockerfile-blossom.integration.ubuntu +++ b/jenkins/Dockerfile-blossom.integration.ubuntu @@ -49,10 +49,11 @@ RUN wget --quiet https://repo.anaconda.com/miniconda/Miniconda3-latest-Linux-x86 ENV PATH="/opt/conda/bin:$MAVEN_HOME/bin:${PATH}" # 'pyarrow' and 'pandas' will be installed as the dependencies of cudf below RUN export CUDA_VER=`echo ${CUDA_VER} | cut -d '.' -f 1,2` && \ - conda install -y -c rapidsai -c rapidsai-nightly -c nvidia -c conda-forge -c defaults cudf=${CUDF_VER} python=3.8 cudatoolkit=${CUDA_VER} && \ - conda install -y spacy && python -m spacy download en_core_web_sm && \ - conda install -y -c anaconda pytest requests && \ - conda install -y -c conda-forge sre_yield && \ + conda install -c conda-forge mamba && \ + mamba install -y -c rapidsai -c rapidsai-nightly -c nvidia -c conda-forge -c defaults cudf=${CUDF_VER} python=3.8 cudatoolkit=${CUDA_VER} && \ + mamba install -y spacy && python -m spacy download en_core_web_sm && \ + mamba install -y -c anaconda pytest requests && \ + mamba install -y -c conda-forge sre_yield && \ conda clean -ay RUN apt install -y inetutils-ping expect From 9308ffdd55b32c2bb9756389e784f0f2a7ca012f Mon Sep 17 00:00:00 2001 From: Alfred Xu Date: Fri, 3 Dec 2021 10:42:09 +0800 Subject: [PATCH 15/52] make up missing ORC write tests on Map of Decimal (#4263) Signed-off-by: sperlingxx --- integration_tests/src/main/python/orc_write_test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py index 83abd44a544..7b68c0a3f9d 100644 --- a/integration_tests/src/main/python/orc_write_test.py +++ b/integration_tests/src/main/python/orc_write_test.py @@ -39,7 +39,9 @@ orc_write_basic_map_gens = [simple_string_to_string_map_gen] + [MapGen(f(nullable=False), f()) for f in [ BooleanGen, ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen, lambda nullable=True: TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc), nullable=nullable), - lambda nullable=True: DateGen(start=date(1590, 1, 1), nullable=nullable)]] + lambda nullable=True: DateGen(start=date(1590, 1, 1), nullable=nullable), + lambda nullable=True: DecimalGen(precision=15, scale=1, nullable=nullable), + lambda nullable=True: DecimalGen(precision=36, scale=5, nullable=nullable)]] orc_write_gens_list = [orc_write_basic_gens, orc_write_struct_gens_sample, From 66d0ab2ac1110005b724de7b57bb07dd6a9f1d04 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Dec 2021 07:06:04 -0700 Subject: [PATCH 16/52] Skip test_regexp_replace_null_pattern_fallback on Spark 3.1.1 and later (#4271) Signed-off-by: Andy Grove --- integration_tests/src/main/python/string_test.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/integration_tests/src/main/python/string_test.py b/integration_tests/src/main/python/string_test.py index 0581250ed74..f4f42924878 100644 --- a/integration_tests/src/main/python/string_test.py +++ b/integration_tests/src/main/python/string_test.py @@ -20,7 +20,7 @@ from marks import * from pyspark.sql.types import * import pyspark.sql.functions as f -from spark_session import is_before_spark_320 +from spark_session import is_before_spark_311, is_before_spark_320 def mk_str_gen(pattern): return StringGen(pattern).with_special_case('').with_special_pattern('.{0,10}') @@ -508,12 +508,15 @@ def test_regexp_like(): @pytest.mark.skipif(is_databricks_runtime(), reason='Databricks optimizes out regexp_replace call in this case') +@pytest.mark.skipif(not is_before_spark_311(), + reason='Spark 3.1.1 optimizes out regexp_replace call in this case') @allow_non_gpu('ProjectExec', 'RegExpReplace') def test_regexp_replace_null_pattern_fallback(): gen = mk_str_gen('[abcd]{0,3}') - # Apache Spark translates `NULL` to `CAST(NULL as STRING)` and we only support + # Spark 3.0.1 translates `NULL` to `CAST(NULL as STRING)` and we only support # literal expressions for the regex pattern - # Databricks Spark replaces the whole regexp_replace expression with a literal null + # Spark 3.1.1 (and Databricks) replaces the whole regexp_replace expression with a + # literal null assert_gpu_fallback_collect( lambda spark: unary_op_df(spark, gen).selectExpr( 'regexp_replace(a, NULL, "A")'), From 8324b700b94193007c9d463a7c6e1366e2c701c4 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 3 Dec 2021 08:18:56 -0600 Subject: [PATCH 17/52] Add tests for timestamps that overflowed before. (#4274) Signed-off-by: Robert (Bobby) Evans --- .../test/scala/com/nvidia/spark/rapids/CastOpSuite.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala index 938b830a6ec..d127b216888 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala @@ -1393,6 +1393,12 @@ object CastOpSuite { def validTimestamps(session: SparkSession): DataFrame = { import session.sqlContext.implicits._ val timestampStrings = Seq( + "8669-07-22T04:45:57.73", + "6233-08-04T19:30:55.701", + "8220-02-25T10:01:15.106", + "9754-01-21T16:53:02.137", + "7649-11-16T15:56:04.996", + "7027-04-09T15:08:52.627", "1920-12-31T11:59:59.999", "1969-12-31T23:59:59.999", "1969-12-31T23:59:59.999999", From 82ca15acbc4626c98d9d16ac22bb17f903a2c783 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Fri, 3 Dec 2021 22:23:36 +0800 Subject: [PATCH 18/52] Commonize v2 shim [databricks] (#4235) * Commonize v2 shim Signed-off-by: Chong Gao * Reuse some codes Signed-off-by: Chong Gao * Revert "Reuse some codes" This reverts commit 25549273d0abd7e1bd6a980e520224b597430a53. * Reuse codes Signed-off-by: Chong Gao * Revert a file Signed-off-by: Chong Gao --- pom.xml | 3 +- .../rapids/shims/spark301/Spark301Shims.scala | 3 +- .../shims/spark301db/Spark301dbShims.scala | 2 +- .../rapids/shims/spark302/Spark302Shims.scala | 2 +- .../rapids/shims/spark303/Spark303Shims.scala | 2 +- .../rapids/shims/spark304/Spark304Shims.scala | 2 +- .../rapids/shims/spark311/Spark311Shims.scala | 2 +- .../shims/spark311cdh/Spark311CDHShims.scala | 2 +- .../rapids/shims/spark312/Spark312Shims.scala | 2 +- .../shims/spark312db/Spark312dbShims.scala | 2 +- .../rapids/shims/spark313/Spark313Shims.scala | 2 +- ...kBaseShims.scala => Spark30XdbShims.scala} | 2 +- ...XShims.scala => Spark30XdbShimsBase.scala} | 2 +- ...arkBaseShims.scala => Spark30XShims.scala} | 365 +------- .../shims/v2/Spark301util320Shims.scala | 455 ++++++++++ .../spark/rapids/shims/v2/Spark30XShims.scala | 145 --- .../spark/rapids/shims/v2/Spark31XShims.scala | 506 ++++++++++- .../rapids/shims/v2/SparkBaseShims.scala | 822 ------------------ .../spark/rapids/shims/v2/AQEUtils.scala | 0 ...kBaseShims.scala => Spark31XdbShims.scala} | 2 +- .../shims/v2/Spark31XdbShimsBase.scala} | 4 +- .../shims/v2/GpuShuffleExchangeExec.scala | 0 22 files changed, 989 insertions(+), 1338 deletions(-) rename sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/{SparkBaseShims.scala => Spark30XdbShims.scala} (99%) rename sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/{Spark30XShims.scala => Spark30XdbShimsBase.scala} (99%) rename sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/{SparkBaseShims.scala => Spark30XShims.scala} (52%) create mode 100644 sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala delete mode 100644 sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala delete mode 100644 sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala rename sql-plugin/src/main/{312db => 31xdb}/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala (100%) rename sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/{SparkBaseShims.scala => Spark31XdbShims.scala} (99%) rename sql-plugin/src/main/{312db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala => 31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShimsBase.scala} (97%) rename sql-plugin/src/main/{312db => 31xdb}/scala/org/apache/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala (100%) diff --git a/pom.xml b/pom.xml index f9a5b92d48b..13aac964269 100644 --- a/pom.xml +++ b/pom.xml @@ -405,7 +405,6 @@ ${project.basedir}/src/main/311+-all/scala ${project.basedir}/src/main/311until320-noncdh/scala ${project.basedir}/src/main/31xdb/scala - ${project.basedir}/src/main/312db/scala ${project.basedir}/src/main/post320-treenode/scala @@ -445,12 +444,12 @@ ${project.basedir}/src/main/301+-nondb/scala ${project.basedir}/src/main/301until320-all/scala ${project.basedir}/src/main/301until320-nondb/scala + ${project.basedir}/src/main/301until330-all/scala ${project.basedir}/src/main/311+-all/scala ${project.basedir}/src/main/311+-nondb/scala ${project.basedir}/src/main/311until320-all/scala ${project.basedir}/src/main/311until320-noncdh/scala ${project.basedir}/src/main/311until320-nondb/scala - ${project.basedir}/src/main/301until330-all/scala ${project.basedir}/src/main/pre320-treenode/scala diff --git a/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/Spark301Shims.scala b/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/Spark301Shims.scala index de376ab7b1e..31b390c7742 100644 --- a/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/Spark301Shims.scala +++ b/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/Spark301Shims.scala @@ -23,8 +23,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.internal.SQLConf -class Spark301Shims extends SparkBaseShims - with Spark30Xuntil33XShims { +class Spark301Shims extends Spark30XShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION diff --git a/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/Spark301dbShims.scala b/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/Spark301dbShims.scala index 0aa24c48b7d..f77e2447ee2 100644 --- a/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/Spark301dbShims.scala +++ b/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/Spark301dbShims.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids.shims.spark301db import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.shims.v2._ -class Spark301dbShims extends SparkBaseShims with Spark30Xuntil33XShims { +class Spark301dbShims extends Spark30XdbShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION } diff --git a/shims/spark302/src/main/scala/com/nvidia/spark/rapids/shims/spark302/Spark302Shims.scala b/shims/spark302/src/main/scala/com/nvidia/spark/rapids/shims/spark302/Spark302Shims.scala index e500de6ca27..8399c2a99f0 100644 --- a/shims/spark302/src/main/scala/com/nvidia/spark/rapids/shims/spark302/Spark302Shims.scala +++ b/shims/spark302/src/main/scala/com/nvidia/spark/rapids/shims/spark302/Spark302Shims.scala @@ -23,7 +23,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.internal.SQLConf -class Spark302Shims extends SparkBaseShims with Spark30Xuntil33XShims { +class Spark302Shims extends Spark30XShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION diff --git a/shims/spark303/src/main/scala/com/nvidia/spark/rapids/shims/spark303/Spark303Shims.scala b/shims/spark303/src/main/scala/com/nvidia/spark/rapids/shims/spark303/Spark303Shims.scala index 844347c12f9..41116562b8e 100644 --- a/shims/spark303/src/main/scala/com/nvidia/spark/rapids/shims/spark303/Spark303Shims.scala +++ b/shims/spark303/src/main/scala/com/nvidia/spark/rapids/shims/spark303/Spark303Shims.scala @@ -23,7 +23,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.internal.SQLConf -class Spark303Shims extends SparkBaseShims with Spark30Xuntil33XShims { +class Spark303Shims extends Spark30XShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION diff --git a/shims/spark304/src/main/scala/com/nvidia/spark/rapids/shims/spark304/Spark304Shims.scala b/shims/spark304/src/main/scala/com/nvidia/spark/rapids/shims/spark304/Spark304Shims.scala index aa477178a19..346d9c2951c 100644 --- a/shims/spark304/src/main/scala/com/nvidia/spark/rapids/shims/spark304/Spark304Shims.scala +++ b/shims/spark304/src/main/scala/com/nvidia/spark/rapids/shims/spark304/Spark304Shims.scala @@ -23,7 +23,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.internal.SQLConf -class Spark304Shims extends SparkBaseShims with Spark30Xuntil33XShims { +class Spark304Shims extends Spark30XShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION diff --git a/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/Spark311Shims.scala b/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/Spark311Shims.scala index d14167a4381..4f7d30a7681 100644 --- a/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/Spark311Shims.scala +++ b/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/Spark311Shims.scala @@ -23,7 +23,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.internal.SQLConf -class Spark311Shims extends SparkBaseShims with Spark30Xuntil33XShims { +class Spark311Shims extends Spark31XShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION diff --git a/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/Spark311CDHShims.scala b/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/Spark311CDHShims.scala index 1f2dc2cb9a5..c687755e4a8 100644 --- a/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/Spark311CDHShims.scala +++ b/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/Spark311CDHShims.scala @@ -23,7 +23,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.internal.SQLConf -class Spark311CDHShims extends SparkBaseShims with Spark30Xuntil33XShims { +class Spark311CDHShims extends Spark31XShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION diff --git a/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/Spark312Shims.scala b/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/Spark312Shims.scala index e561b8e56eb..5ada5b10c0a 100644 --- a/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/Spark312Shims.scala +++ b/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/Spark312Shims.scala @@ -23,7 +23,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.internal.SQLConf -class Spark312Shims extends SparkBaseShims with Spark30Xuntil33XShims { +class Spark312Shims extends Spark31XShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION diff --git a/shims/spark312db/src/main/scala/com/nvidia/spark/rapids/shims/spark312db/Spark312dbShims.scala b/shims/spark312db/src/main/scala/com/nvidia/spark/rapids/shims/spark312db/Spark312dbShims.scala index 64340b943d1..933994f2256 100644 --- a/shims/spark312db/src/main/scala/com/nvidia/spark/rapids/shims/spark312db/Spark312dbShims.scala +++ b/shims/spark312db/src/main/scala/com/nvidia/spark/rapids/shims/spark312db/Spark312dbShims.scala @@ -23,7 +23,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.internal.SQLConf -class Spark312dbShims extends SparkBaseShims with Spark30Xuntil33XShims { +class Spark312dbShims extends Spark31XdbShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION diff --git a/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/Spark313Shims.scala b/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/Spark313Shims.scala index e87d9228df0..020a3b74027 100644 --- a/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/Spark313Shims.scala +++ b/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/Spark313Shims.scala @@ -23,7 +23,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.internal.SQLConf -class Spark313Shims extends SparkBaseShims with Spark30Xuntil33XShims { +class Spark313Shims extends Spark31XShims with Spark30Xuntil33XShims { override def getSparkShimVersion: ShimVersion = SparkShimServiceProvider.VERSION diff --git a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala similarity index 99% rename from sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala rename to sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala index 8980be109aa..66b32106c1c 100644 --- a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala +++ b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala @@ -68,7 +68,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManagerId} import org.apache.spark.unsafe.types.CalendarInterval -abstract class SparkBaseShims extends Spark30XShims with Logging { +abstract class Spark30XdbShims extends Spark30XdbShimsBase with Logging { override def getParquetFilters( schema: MessageType, pushDownDate: Boolean, diff --git a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShimsBase.scala similarity index 99% rename from sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala rename to sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShimsBase.scala index 7015685515d..ebb54757e8b 100644 --- a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala +++ b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShimsBase.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.rapids.execution.GpuCustomShuffleReaderExec /** * Shim base class that can be compiled with every supported 3.0.x */ -trait Spark30XShims extends SparkShims { +trait Spark30XdbShimsBase extends SparkShims { override def parquetRebaseReadKey: String = SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key override def parquetRebaseWriteKey: String = diff --git a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala similarity index 52% rename from sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala rename to sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala index 9795cc56725..a40fd16ff1b 100644 --- a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala +++ b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala @@ -16,59 +16,38 @@ package com.nvidia.spark.rapids.shims.v2 -import java.net.URI import java.nio.ByteBuffer -import com.esotericsoftware.kryo.Kryo -import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.nvidia.spark.rapids._ import org.apache.arrow.memory.ReferenceManager import org.apache.arrow.vector.ValueVector -import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.rapids.shims.v2.GpuShuffleExchangeExec -import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Average -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.command.{AlterTableRecoverPartitionsCommand, RunnableCommand} -import org.apache.spark.sql.execution.datasources.{FileIndex, FilePartition, FileScanRDD, HadoopFsRelation, InMemoryFileIndex, PartitionDirectory, PartitionedFile, PartitioningAwareFileIndex} -import org.apache.spark.sql.execution.datasources.rapids.GpuPartitioningUtils -import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan -import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeExec} -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex} +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.python.{AggregateInPandasExec, ArrowEvalPythonExec, FlatMapGroupsInPandasExec, MapInPandasExec, WindowInPandasExec} -import org.apache.spark.sql.execution.window.WindowExecBase import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.{GpuAbs, GpuAverage, GpuFileSourceScanExec, GpuTimeSub} -import org.apache.spark.sql.rapids.execution.{GpuBroadcastNestedLoopJoinExecBase, GpuShuffleExchangeExecBase, JoinTypeChecks, SerializeBatchDeserializeHostBuffer, SerializeConcatHostBuffersDeserializeBatch} +import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBase, JoinTypeChecks} import org.apache.spark.sql.rapids.execution.python._ import org.apache.spark.sql.rapids.execution.python.shims.v2._ -import org.apache.spark.sql.rapids.shims.v2.GpuSchemaUtils -import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManagerId} import org.apache.spark.unsafe.types.CalendarInterval -abstract class SparkBaseShims extends Spark30XShims with Logging { - - override def v1RepairTableCommand(tableName: TableIdentifier): RunnableCommand = - AlterTableRecoverPartitionsCommand(tableName) - +abstract class Spark30XShims extends Spark301util320Shims with Logging { override def getScalaUDFAsExpression( function: AnyRef, dataType: DataType, @@ -92,16 +71,6 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { startMapIndex, endMapIndex, startPartition, endPartition) } - override def getGpuBroadcastNestedLoopJoinShim( - left: SparkPlan, - right: SparkPlan, - join: BroadcastNestedLoopJoinExec, - joinType: JoinType, - condition: Option[Expression], - targetSizeBytes: Long): GpuBroadcastNestedLoopJoinExecBase = { - GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition, targetSizeBytes) - } - override def getGpuShuffleExchangeExec( gpuOutputPartitioning: GpuPartitioning, child: SparkPlan, @@ -117,22 +86,6 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { queryStage.shuffle.asInstanceOf[GpuShuffleExchangeExecBase] } - override def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = { - plan match { - case _: GpuBroadcastHashJoinExec => true - case _ => false - } - } - - override def isWindowFunctionExec(plan: SparkPlan): Boolean = plan.isInstanceOf[WindowExecBase] - - override def isGpuShuffledHashJoin(plan: SparkPlan): Boolean = { - plan match { - case _: GpuShuffledHashJoinExec => true - case _ => false - } - } - override def getExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = { Seq( GpuOverrides.exec[WindowInPandasExec]( @@ -159,7 +112,7 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { GpuOverrides.exec[FileSourceScanExec]( "Reading data from files, often from Hive tables", ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all), + TypeSig.ARRAY + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all), (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { // partition filters and data filters are not run on the GPU @@ -210,8 +163,8 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), GpuOverrides.exec[ArrowEvalPythonExec]( "The backend of the Scalar Pandas UDFs. Accelerates the data transfer between the" + - " Java process and the Python process. It also supports scheduling GPU resources" + - " for the Python process when enabled", + " Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled", ExecChecks( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), TypeSig.all), @@ -235,21 +188,21 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { }), GpuOverrides.exec[MapInPandasExec]( "The backend for Map Pandas Iterator UDF. Accelerates the data transfer between the" + - " Java process and the Python process. It also supports scheduling GPU resources" + - " for the Python process when enabled.", + " Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled.", ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), TypeSig.all), (mapPy, conf, p, r) => new GpuMapInPandasExecMeta(mapPy, conf, p, r)), GpuOverrides.exec[FlatMapGroupsInPandasExec]( "The backend for Flat Map Groups Pandas UDF, Accelerates the data transfer between the" + - " Java process and the Python process. It also supports scheduling GPU resources" + - " for the Python process when enabled.", + " Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled.", ExecChecks(TypeSig.commonCudfTypes, TypeSig.all), (flatPy, conf, p, r) => new GpuFlatMapGroupsInPandasExecMeta(flatPy, conf, p, r)), GpuOverrides.exec[AggregateInPandasExec]( "The backend for an Aggregation Pandas UDF, this accelerates the data transfer between" + - " the Java process and the Python process. It also supports scheduling GPU resources" + - " for the Python process when enabled.", + " the Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled.", ExecChecks(TypeSig.commonCudfTypes, TypeSig.all), (aggPy, conf, p, r) => new GpuAggregateInPandasExecMeta(aggPy, conf, p, r)) ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap @@ -273,7 +226,7 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { ExprChecks.fullAgg( TypeSig.DOUBLE + TypeSig.DECIMAL_128_FULL, TypeSig.DOUBLE + TypeSig.DECIMAL_128_FULL, - Seq(ParamCheck("input", + Seq(ParamCheck("input", TypeSig.integral + TypeSig.fp + TypeSig.DECIMAL_128_FULL, TypeSig.numeric))), (a, conf, p, r) => new AggExprMeta[Average](a, conf, p, r) { @@ -288,11 +241,11 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { if (dt.precision > 23) { if (conf.needDecimalGuarantees) { willNotWorkOnGpu("GpuAverage cannot guarantee proper overflow checks for " + - s"a precision large than 23. The current precision is ${dt.precision}") + s"a precision large than 23. The current precision is ${dt.precision}") } else { logWarning("Decimal overflow guarantees disabled for " + - s"Average(${a.child.dataType}) produces $dt with an " + - s"intermediate precision of ${dt.precision + 15}") + s"Average(${a.child.dataType}) produces $dt with an " + + s"intermediate precision of ${dt.precision + 15}") } } case _ => // NOOP @@ -334,7 +287,7 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { ExprChecks.binaryProject(TypeSig.TIMESTAMP, TypeSig.TIMESTAMP, ("start", TypeSig.TIMESTAMP, TypeSig.TIMESTAMP), ("interval", TypeSig.lit(TypeEnum.CALENDAR) - .withPsNote(TypeEnum.CALENDAR, "months not supported"), TypeSig.CALENDAR)), + .withPsNote(TypeEnum.CALENDAR, "months not supported"), TypeSig.CALENDAR)), (timeSub, conf, p, r) => new BinaryExprMeta[TimeSub](timeSub, conf, p, r) { override def tagExprForGpu(): Unit = { timeSub.interval match { @@ -352,141 +305,14 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { })) } - override def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = Seq( - GpuOverrides.scan[ParquetScan]( - "Parquet parsing", - (a, conf, p, r) => new ScanMeta[ParquetScan](a, conf, p, r) { - override def tagSelfForGpu(): Unit = GpuParquetScanBase.tagSupport(this) - - override def convertToGpu(): Scan = { - GpuParquetScan(a.sparkSession, - a.hadoopConf, - a.fileIndex, - a.dataSchema, - a.readDataSchema, - a.readPartitionSchema, - a.pushedFilters, - a.options, - a.partitionFilters, - a.dataFilters, - conf) - } - }), - GpuOverrides.scan[OrcScan]( - "ORC parsing", - (a, conf, p, r) => new ScanMeta[OrcScan](a, conf, p, r) { - override def tagSelfForGpu(): Unit = - GpuOrcScanBase.tagSupport(this) - - override def convertToGpu(): Scan = - GpuOrcScan(a.sparkSession, - a.hadoopConf, - a.fileIndex, - a.dataSchema, - a.readDataSchema, - a.readPartitionSchema, - a.options, - a.pushedFilters, - a.partitionFilters, - a.dataFilters, - conf) - }) - ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap - - override def getBuildSide(join: HashJoin): GpuBuildSide = { - GpuJoinUtils.getGpuBuildSide(join.buildSide) - } - - override def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { - GpuJoinUtils.getGpuBuildSide(join.buildSide) - } - - override def getPartitionFileNames( - partitions: Seq[PartitionDirectory]): Seq[String] = { - val files = partitions.flatMap(partition => partition.files) - files.map(_.getPath.getName) - } - - override def getPartitionFileStatusSize(partitions: Seq[PartitionDirectory]): Long = { - partitions.map(_.files.map(_.getLen).sum).sum - } - - override def getPartitionedFiles( - partitions: Array[PartitionDirectory]): Array[PartitionedFile] = { - partitions.flatMap { p => - p.files.map { f => - PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values) - } - } - } - - override def getPartitionSplitFiles( - partitions: Array[PartitionDirectory], - maxSplitBytes: Long, - relation: HadoopFsRelation): Array[PartitionedFile] = { - partitions.flatMap { partition => - partition.files.flatMap { file => - // getPath() is very expensive so we only want to call it once in this block: - val filePath = file.getPath - val isSplitable = relation.fileFormat.isSplitable( - relation.sparkSession, relation.options, filePath) - PartitionedFileUtil.splitFiles( - sparkSession = relation.sparkSession, - file = file, - filePath = filePath, - isSplitable = isSplitable, - maxSplitBytes = maxSplitBytes, - partitionValues = partition.values - ) - } - } - } - - override def getFileScanRDD( - sparkSession: SparkSession, - readFunction: PartitionedFile => Iterator[InternalRow], - filePartitions: Seq[FilePartition]): RDD[InternalRow] = { - new FileScanRDD(sparkSession, readFunction, filePartitions) - } - // Hardcoded for Spark-3.0.* override def getFileSourceMaxMetadataValueLength(sqlConf: SQLConf): Int = 100 - override def createFilePartition(index: Int, files: Array[PartitionedFile]): FilePartition = { - FilePartition(index, files) - } - - override def copyBatchScanExec( - batchScanExec: GpuBatchScanExec, - queryUsesInputFile: Boolean): GpuBatchScanExec = { - val scanCopy = batchScanExec.scan match { - case parquetScan: GpuParquetScan => - parquetScan.copy(queryUsesInputFile=queryUsesInputFile) - case orcScan: GpuOrcScan => - orcScan.copy(queryUsesInputFile=queryUsesInputFile) - case _ => throw new RuntimeException("Wrong format") // never reach here - } - batchScanExec.copy(scan=scanCopy) - } - - override def copyFileSourceScanExec( - scanExec: GpuFileSourceScanExec, - queryUsesInputFile: Boolean): GpuFileSourceScanExec = { - scanExec.copy(queryUsesInputFile=queryUsesInputFile)(scanExec.rapidsConf) - } - override def getGpuColumnarToRowTransition(plan: SparkPlan, - exportColumnRdd: Boolean): GpuColumnarToRowExecParent = { + exportColumnRdd: Boolean): GpuColumnarToRowExecParent = { GpuColumnarToRowExec(plan, exportColumnRdd) } - override def checkColumnNameDuplication( - schema: StructType, - colType: String, - resolver: Resolver): Unit = { - GpuSchemaUtils.checkColumnNameDuplication(schema, colType, resolver) - } - override def sortOrder( child: Expression, direction: SortDirection, @@ -496,13 +322,6 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { s.copy(child = child) } - override def alias(child: Expression, name: String)( - exprId: ExprId, - qualifier: Seq[String], - explicitMetadata: Option[Metadata]): Alias = { - Alias(child, name)(exprId, qualifier, explicitMetadata) - } - override def shouldIgnorePath(path: String): Boolean = { InMemoryFileIndex.shouldFilterOut(path) } @@ -515,153 +334,19 @@ abstract class SparkBaseShims extends Spark30XShims with Logging { (arrowBuf.nioBuffer(), arrowBuf.getReferenceManager) } - override def getArrowValidityBuf(vec: ValueVector): (ByteBuffer, ReferenceManager) = { - val arrowBuf = vec.getValidityBuffer - (arrowBuf.nioBuffer(), arrowBuf.getReferenceManager) - } - - override def getArrowOffsetsBuf(vec: ValueVector): (ByteBuffer, ReferenceManager) = { - val arrowBuf = vec.getOffsetBuffer - (arrowBuf.nioBuffer(), arrowBuf.getReferenceManager) - } - - override def replaceWithAlluxioPathIfNeeded( - conf: RapidsConf, - relation: HadoopFsRelation, - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]): FileIndex = { - - val alluxioPathsReplace: Option[Seq[String]] = conf.getAlluxioPathsToReplace - - if (alluxioPathsReplace.isDefined) { - // alluxioPathsReplace: Seq("key->value", "key1->value1") - // turn the rules to the Map with eg - // { s3:/foo -> alluxio://0.1.2.3:19998/foo, - // gs:/bar -> alluxio://0.1.2.3:19998/bar, - // /baz -> alluxio://0.1.2.3:19998/baz } - val replaceMapOption = alluxioPathsReplace.map(rules => { - rules.map(rule => { - val split = rule.split("->") - if (split.size == 2) { - split(0).trim -> split(1).trim - } else { - throw new IllegalArgumentException(s"Invalid setting for " + - s"${RapidsConf.ALLUXIO_PATHS_REPLACE.key}") - } - }).toMap - }) - - replaceMapOption.map(replaceMap => { - - def isDynamicPruningFilter(e: Expression): Boolean = - e.find(_.isInstanceOf[PlanExpression[_]]).isDefined - - val partitionDirs = relation.location.listFiles( - partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) - - // replacement func to check if the file path is prefixed with the string user configured - // if yes, replace it - val replaceFunc = (f: Path) => { - val pathStr = f.toString - val matchedSet = replaceMap.keySet.filter(reg => pathStr.startsWith(reg)) - if (matchedSet.size > 1) { - // never reach here since replaceMap is a Map - throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + - s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule for each " + - s"file path") - } else if (matchedSet.size == 1) { - new Path(pathStr.replaceFirst(matchedSet.head, replaceMap(matchedSet.head))) - } else { - f - } - } - - // replace all of input files - val inputFiles: Seq[Path] = partitionDirs.flatMap(partitionDir => { - replacePartitionDirectoryFiles(partitionDir, replaceFunc) - }) - - // replace all of rootPaths which are already unique - val rootPaths = relation.location.rootPaths.map(replaceFunc) - - val parameters: Map[String, String] = relation.options - - // infer PartitionSpec - val partitionSpec = GpuPartitioningUtils.inferPartitioning( - relation.sparkSession, - rootPaths, - inputFiles, - parameters, - Option(relation.dataSchema), - replaceFunc) - - // generate a new InMemoryFileIndex holding paths with alluxio schema - new InMemoryFileIndex( - relation.sparkSession, - inputFiles, - parameters, - Option(relation.dataSchema), - userSpecifiedPartitionSpec = Some(partitionSpec)) - }).getOrElse(relation.location) - - } else { - relation.location - } - } - - override def replacePartitionDirectoryFiles(partitionDir: PartitionDirectory, - replaceFunc: Path => Path): Seq[Path] = { - partitionDir.files.map(f => replaceFunc(f.getPath)) - } - override def shouldFailDivByZero(): Boolean = false - override def reusedExchangeExecPfn: PartialFunction[SparkPlan, ReusedExchangeExec] = { - case ShuffleQueryStageExec(_, e: ReusedExchangeExec) => e - case BroadcastQueryStageExec(_, e: ReusedExchangeExec) => e - } - /** dropped by SPARK-34234 */ override def attachTreeIfSupported[TreeType <: TreeNode[_], A]( - tree: TreeType, - msg: String)( - f: => A + tree: TreeType, + msg: String)( + f: => A ): A = { attachTree(tree, msg)(f) } - override def createTable(table: CatalogTable, - sessionCatalog: SessionCatalog, - tableLocation: Option[URI], - result: BaseRelation) = { - val newTable = table.copy( - storage = table.storage.copy(locationUri = tableLocation), - // We will use the schema of resolved.relation as the schema of the table (instead of - // the schema of df). It is important since the nullability may be changed by the relation - // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). - schema = result.schema) - // Table location is already validated. No need to check it again during table creation. - sessionCatalog.createTable(newTable, ignoreIfExists = false, validateLocation = false) - } - - override def hasAliasQuoteFix: Boolean = false - override def hasCastFloatTimestampUpcast: Boolean = false - override def filesFromFileIndex(fileIndex: PartitioningAwareFileIndex): Seq[FileStatus] = { - fileIndex.allFiles() - } - - override def broadcastModeTransform(mode: BroadcastMode, rows: Array[InternalRow]): Any = - mode.transform(rows) - - override def registerKryoClasses(kryo: Kryo): Unit = { - kryo.register(classOf[SerializeConcatHostBuffersDeserializeBatch], - new KryoJavaSerializer()) - kryo.register(classOf[SerializeBatchDeserializeHostBuffer], - new KryoJavaSerializer()) - } - override def getAdaptiveInputPlan(adaptivePlan: AdaptiveSparkPlanExec): SparkPlan = { adaptivePlan.initialPlan } diff --git a/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala b/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala new file mode 100644 index 00000000000..457dcc650c0 --- /dev/null +++ b/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala @@ -0,0 +1,455 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.v2 + +import java.net.URI +import java.nio.ByteBuffer + +import scala.collection.mutable.ListBuffer + +import com.esotericsoftware.kryo.Kryo +import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuOverrides.exec +import org.apache.arrow.memory.ReferenceManager +import org.apache.arrow.vector.ValueVector +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils} +import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec, CustomShuffleReaderExec, QueryStageExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.command.{AlterTableRecoverPartitionsCommand, RunnableCommand} +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.rapids.GpuPartitioningUtils +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.window.WindowExecBase +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids._ +import org.apache.spark.sql.rapids.execution.{GpuBroadcastNestedLoopJoinExecBase, GpuCustomShuffleReaderExec, SerializeBatchDeserializeHostBuffer, SerializeConcatHostBuffersDeserializeBatch} +import org.apache.spark.sql.rapids.shims.v2.GpuSchemaUtils +import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.sql.types._ + +/** +* Shim base class that can be compiled with from 301 until 320 +*/ +trait Spark301util320Shims extends SparkShims { + override def parquetRebaseReadKey: String = + SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key + override def parquetRebaseWriteKey: String = + SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key + override def avroRebaseReadKey: String = + SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key + override def avroRebaseWriteKey: String = + SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key + override def parquetRebaseRead(conf: SQLConf): String = + conf.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ) + override def parquetRebaseWrite(conf: SQLConf): String = + conf.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE) + override def int96ParquetRebaseRead(conf: SQLConf): String = + parquetRebaseRead(conf) + override def int96ParquetRebaseWrite(conf: SQLConf): String = + parquetRebaseWrite(conf) + override def int96ParquetRebaseReadKey: String = + parquetRebaseReadKey + override def int96ParquetRebaseWriteKey: String = + parquetRebaseWriteKey + override def hasSeparateINT96RebaseConf: Boolean = false + + override def sessionFromPlan(plan: SparkPlan): SparkSession = { + plan.sqlContext.sparkSession + } + + override def filesFromFileIndex(fileIndex: PartitioningAwareFileIndex): Seq[FileStatus] = { + fileIndex.allFiles() + } + + def broadcastModeTransform(mode: BroadcastMode, rows: Array[InternalRow]): Any = + mode.transform(rows) + + override def newBroadcastQueryStageExec( + old: BroadcastQueryStageExec, + newPlan: SparkPlan): BroadcastQueryStageExec = BroadcastQueryStageExec(old.id, newPlan) + + override def getDateFormatter(): DateFormatter = { + DateFormatter(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + } + + override def isExchangeOp(plan: SparkPlanMeta[_]): Boolean = { + // if the child query stage already executed on GPU then we need to keep the + // next operator on GPU in these cases + SQLConf.get.adaptiveExecutionEnabled && (plan.wrapped match { + case _: CustomShuffleReaderExec + | _: ShuffledHashJoinExec + | _: BroadcastHashJoinExec + | _: BroadcastExchangeExec + | _: BroadcastNestedLoopJoinExec => true + case _ => false + }) + } + + override def isAqePlan(p: SparkPlan): Boolean = p match { + case _: AdaptiveSparkPlanExec | + _: QueryStageExec | + _: CustomShuffleReaderExec => true + case _ => false + } + + override def isCustomReaderExec(x: SparkPlan): Boolean = x match { + case _: GpuCustomShuffleReaderExec | _: CustomShuffleReaderExec => true + case _ => false + } + + override def aqeShuffleReaderExec: ExecRule[_ <: SparkPlan] = exec[CustomShuffleReaderExec]( + "A wrapper of shuffle query stage", + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL + TypeSig.ARRAY + + TypeSig.STRUCT + TypeSig.MAP).nested(), TypeSig.all), + (exec, conf, p, r) => new GpuCustomShuffleReaderMeta(exec, conf, p, r)) + + override def findOperators(plan: SparkPlan, predicate: SparkPlan => Boolean): Seq[SparkPlan] = { + def recurse( + plan: SparkPlan, + predicate: SparkPlan => Boolean, + accum: ListBuffer[SparkPlan]): Seq[SparkPlan] = { + if (predicate(plan)) { + accum += plan + } + plan match { + case a: AdaptiveSparkPlanExec => recurse(a.executedPlan, predicate, accum) + case qs: BroadcastQueryStageExec => recurse(qs.broadcast, predicate, accum) + case qs: ShuffleQueryStageExec => recurse(qs.shuffle, predicate, accum) + case other => other.children.flatMap(p => recurse(p, predicate, accum)).headOption + } + accum + } + recurse(plan, predicate, new ListBuffer[SparkPlan]()) + } + + override def skipAssertIsOnTheGpu(plan: SparkPlan): Boolean = false + + override def shouldFailDivOverflow(): Boolean = false + + override def leafNodeDefaultParallelism(ss: SparkSession): Int = { + ss.sparkContext.defaultParallelism + } + + override def shouldFallbackOnAnsiTimestamp(): Boolean = false + + override def getLegacyStatisticalAggregate(): Boolean = true + + override def v1RepairTableCommand(tableName: TableIdentifier): RunnableCommand = + AlterTableRecoverPartitionsCommand(tableName) + + override def getGpuBroadcastNestedLoopJoinShim( + left: SparkPlan, + right: SparkPlan, + join: BroadcastNestedLoopJoinExec, + joinType: JoinType, + condition: Option[Expression], + targetSizeBytes: Long): GpuBroadcastNestedLoopJoinExecBase = { + GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition, targetSizeBytes) + } + + override def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = { + plan match { + case _: GpuBroadcastHashJoinExec => true + case _ => false + } + } + + override def isWindowFunctionExec(plan: SparkPlan): Boolean = plan.isInstanceOf[WindowExecBase] + + override def isGpuShuffledHashJoin(plan: SparkPlan): Boolean = { + plan match { + case _: GpuShuffledHashJoinExec => true + case _ => false + } + } + + override def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = Seq( + GpuOverrides.scan[ParquetScan]( + "Parquet parsing", + (a, conf, p, r) => new ScanMeta[ParquetScan](a, conf, p, r) { + override def tagSelfForGpu(): Unit = GpuParquetScanBase.tagSupport(this) + + override def convertToGpu(): Scan = { + GpuParquetScan(a.sparkSession, + a.hadoopConf, + a.fileIndex, + a.dataSchema, + a.readDataSchema, + a.readPartitionSchema, + a.pushedFilters, + a.options, + a.partitionFilters, + a.dataFilters, + conf) + } + }), + GpuOverrides.scan[OrcScan]( + "ORC parsing", + (a, conf, p, r) => new ScanMeta[OrcScan](a, conf, p, r) { + override def tagSelfForGpu(): Unit = + GpuOrcScanBase.tagSupport(this) + + override def convertToGpu(): Scan = + GpuOrcScan(a.sparkSession, + a.hadoopConf, + a.fileIndex, + a.dataSchema, + a.readDataSchema, + a.readPartitionSchema, + a.options, + a.pushedFilters, + a.partitionFilters, + a.dataFilters, + conf) + }) + ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap + + override def getBuildSide(join: HashJoin): GpuBuildSide = { + GpuJoinUtils.getGpuBuildSide(join.buildSide) + } + + override def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { + GpuJoinUtils.getGpuBuildSide(join.buildSide) + } + + override def getPartitionFileNames( + partitions: Seq[PartitionDirectory]): Seq[String] = { + val files = partitions.flatMap(partition => partition.files) + files.map(_.getPath.getName) + } + + override def getPartitionFileStatusSize(partitions: Seq[PartitionDirectory]): Long = { + partitions.map(_.files.map(_.getLen).sum).sum + } + + override def getPartitionedFiles( + partitions: Array[PartitionDirectory]): Array[PartitionedFile] = { + partitions.flatMap { p => + p.files.map { f => + PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values) + } + } + } + + override def getPartitionSplitFiles( + partitions: Array[PartitionDirectory], + maxSplitBytes: Long, + relation: HadoopFsRelation): Array[PartitionedFile] = { + partitions.flatMap { partition => + partition.files.flatMap { file => + // getPath() is very expensive so we only want to call it once in this block: + val filePath = file.getPath + val isSplitable = relation.fileFormat.isSplitable( + relation.sparkSession, relation.options, filePath) + PartitionedFileUtil.splitFiles( + sparkSession = relation.sparkSession, + file = file, + filePath = filePath, + isSplitable = isSplitable, + maxSplitBytes = maxSplitBytes, + partitionValues = partition.values + ) + } + } + } + + override def getFileScanRDD( + sparkSession: SparkSession, + readFunction: PartitionedFile => Iterator[InternalRow], + filePartitions: Seq[FilePartition]): RDD[InternalRow] = { + new FileScanRDD(sparkSession, readFunction, filePartitions) + } + + override def createFilePartition(index: Int, files: Array[PartitionedFile]): FilePartition = { + FilePartition(index, files) + } + + override def copyBatchScanExec( + batchScanExec: GpuBatchScanExec, + queryUsesInputFile: Boolean): GpuBatchScanExec = { + val scanCopy = batchScanExec.scan match { + case parquetScan: GpuParquetScan => + parquetScan.copy(queryUsesInputFile = queryUsesInputFile) + case orcScan: GpuOrcScan => + orcScan.copy(queryUsesInputFile = queryUsesInputFile) + case _ => throw new RuntimeException("Wrong format") // never reach here + } + batchScanExec.copy(scan = scanCopy) + } + + override def copyFileSourceScanExec( + scanExec: GpuFileSourceScanExec, + queryUsesInputFile: Boolean): GpuFileSourceScanExec = { + scanExec.copy(queryUsesInputFile = queryUsesInputFile)(scanExec.rapidsConf) + } + + override def checkColumnNameDuplication( + schema: StructType, + colType: String, + resolver: Resolver): Unit = { + GpuSchemaUtils.checkColumnNameDuplication(schema, colType, resolver) + } + + override def alias(child: Expression, name: String)( + exprId: ExprId, + qualifier: Seq[String], + explicitMetadata: Option[Metadata]): Alias = { + Alias(child, name)(exprId, qualifier, explicitMetadata) + } + + override def getArrowValidityBuf(vec: ValueVector): (ByteBuffer, ReferenceManager) = { + val arrowBuf = vec.getValidityBuffer + (arrowBuf.nioBuffer(), arrowBuf.getReferenceManager) + } + + override def getArrowOffsetsBuf(vec: ValueVector): (ByteBuffer, ReferenceManager) = { + val arrowBuf = vec.getOffsetBuffer + (arrowBuf.nioBuffer(), arrowBuf.getReferenceManager) + } + + override def replaceWithAlluxioPathIfNeeded( + conf: RapidsConf, + relation: HadoopFsRelation, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): FileIndex = { + + val alluxioPathsReplace: Option[Seq[String]] = conf.getAlluxioPathsToReplace + + if (alluxioPathsReplace.isDefined) { + // alluxioPathsReplace: Seq("key->value", "key1->value1") + // turn the rules to the Map with eg + // { s3:/foo -> alluxio://0.1.2.3:19998/foo, + // gs:/bar -> alluxio://0.1.2.3:19998/bar, + // /baz -> alluxio://0.1.2.3:19998/baz } + val replaceMapOption = alluxioPathsReplace.map(rules => { + rules.map(rule => { + val split = rule.split("->") + if (split.size == 2) { + split(0).trim -> split(1).trim + } else { + throw new IllegalArgumentException(s"Invalid setting for " + + s"${RapidsConf.ALLUXIO_PATHS_REPLACE.key}") + } + }).toMap + }) + + replaceMapOption.map(replaceMap => { + + def isDynamicPruningFilter(e: Expression): Boolean = + e.find(_.isInstanceOf[PlanExpression[_]]).isDefined + + val partitionDirs = relation.location.listFiles( + partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) + + // replacement func to check if the file path is prefixed with the string user configured + // if yes, replace it + val replaceFunc = (f: Path) => { + val pathStr = f.toString + val matchedSet = replaceMap.keySet.filter(reg => pathStr.startsWith(reg)) + if (matchedSet.size > 1) { + // never reach here since replaceMap is a Map + throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + + s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule for each " + + s"file path") + } else if (matchedSet.size == 1) { + new Path(pathStr.replaceFirst(matchedSet.head, replaceMap(matchedSet.head))) + } else { + f + } + } + + // replace all of input files + val inputFiles: Seq[Path] = partitionDirs.flatMap(partitionDir => { + replacePartitionDirectoryFiles(partitionDir, replaceFunc) + }) + + // replace all of rootPaths which are already unique + val rootPaths = relation.location.rootPaths.map(replaceFunc) + + val parameters: Map[String, String] = relation.options + + // infer PartitionSpec + val partitionSpec = GpuPartitioningUtils.inferPartitioning( + relation.sparkSession, + rootPaths, + inputFiles, + parameters, + Option(relation.dataSchema), + replaceFunc) + + // generate a new InMemoryFileIndex holding paths with alluxio schema + new InMemoryFileIndex( + relation.sparkSession, + inputFiles, + parameters, + Option(relation.dataSchema), + userSpecifiedPartitionSpec = Some(partitionSpec)) + }).getOrElse(relation.location) + + } else { + relation.location + } + } + + override def replacePartitionDirectoryFiles(partitionDir: PartitionDirectory, + replaceFunc: Path => Path): Seq[Path] = { + partitionDir.files.map(f => replaceFunc(f.getPath)) + } + + override def hasAliasQuoteFix: Boolean = false + + override def registerKryoClasses(kryo: Kryo): Unit = { + kryo.register(classOf[SerializeConcatHostBuffersDeserializeBatch], + new KryoJavaSerializer()) + kryo.register(classOf[SerializeBatchDeserializeHostBuffer], + new KryoJavaSerializer()) + } + + override def reusedExchangeExecPfn: PartialFunction[SparkPlan, ReusedExchangeExec] = { + case ShuffleQueryStageExec(_, e: ReusedExchangeExec) => e + case BroadcastQueryStageExec(_, e: ReusedExchangeExec) => e + } + + override def createTable(table: CatalogTable, + sessionCatalog: SessionCatalog, + tableLocation: Option[URI], + result: BaseRelation) = { + val newTable = table.copy( + storage = table.storage.copy(locationUri = tableLocation), + // We will use the schema of resolved.relation as the schema of the table (instead of + // the schema of df). It is important since the nullability may be changed by the relation + // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). + schema = result.schema) + // Table location is already validated. No need to check it again during table creation. + sessionCatalog.createTable(newTable, ignoreIfExists = false, validateLocation = false) + } +} diff --git a/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala b/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala deleted file mode 100644 index 0b85284a3fc..00000000000 --- a/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Copyright (c) 2021, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids.shims.v2 - -import scala.collection.mutable.ListBuffer - -import com.nvidia.spark.rapids.{ExecChecks, ExecRule, SparkPlanMeta, SparkShims, TypeSig} -import com.nvidia.spark.rapids.GpuOverrides.exec -import org.apache.hadoop.fs.FileStatus - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode -import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils} -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec, CustomShuffleReaderExec, QueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, ShuffledHashJoinExec} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.GpuCustomShuffleReaderExec - -/** -* Shim base class that can be compiled with every supported 3.0.x -*/ -trait Spark30XShims extends SparkShims { - override def parquetRebaseReadKey: String = - SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key - override def parquetRebaseWriteKey: String = - SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key - override def avroRebaseReadKey: String = - SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key - override def avroRebaseWriteKey: String = - SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key - override def parquetRebaseRead(conf: SQLConf): String = - conf.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ) - override def parquetRebaseWrite(conf: SQLConf): String = - conf.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE) - override def int96ParquetRebaseRead(conf: SQLConf): String = - parquetRebaseRead(conf) - override def int96ParquetRebaseWrite(conf: SQLConf): String = - parquetRebaseWrite(conf) - override def int96ParquetRebaseReadKey: String = - parquetRebaseReadKey - override def int96ParquetRebaseWriteKey: String = - parquetRebaseWriteKey - override def hasSeparateINT96RebaseConf: Boolean = false - - override def sessionFromPlan(plan: SparkPlan): SparkSession = { - plan.sqlContext.sparkSession - } - - override def filesFromFileIndex( - fileIndex: PartitioningAwareFileIndex - ): Seq[FileStatus] = { - fileIndex.allFiles() - } - - def broadcastModeTransform(mode: BroadcastMode, rows: Array[InternalRow]): Any = - mode.transform(rows) - - override def newBroadcastQueryStageExec( - old: BroadcastQueryStageExec, - newPlan: SparkPlan): BroadcastQueryStageExec = BroadcastQueryStageExec(old.id, newPlan) - - override def getDateFormatter(): DateFormatter = { - DateFormatter(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) - } - - override def isExchangeOp(plan: SparkPlanMeta[_]): Boolean = { - // if the child query stage already executed on GPU then we need to keep the - // next operator on GPU in these cases - SQLConf.get.adaptiveExecutionEnabled && (plan.wrapped match { - case _: CustomShuffleReaderExec - | _: ShuffledHashJoinExec - | _: BroadcastHashJoinExec - | _: BroadcastExchangeExec - | _: BroadcastNestedLoopJoinExec => true - case _ => false - }) - } - - override def isAqePlan(p: SparkPlan): Boolean = p match { - case _: AdaptiveSparkPlanExec | - _: QueryStageExec | - _: CustomShuffleReaderExec => true - case _ => false - } - - override def isCustomReaderExec(x: SparkPlan): Boolean = x match { - case _: GpuCustomShuffleReaderExec | _: CustomShuffleReaderExec => true - case _ => false - } - - override def aqeShuffleReaderExec: ExecRule[_ <: SparkPlan] = exec[CustomShuffleReaderExec]( - "A wrapper of shuffle query stage", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.MAP).nested(), TypeSig.all), - (exec, conf, p, r) => new GpuCustomShuffleReaderMeta(exec, conf, p, r)) - - override def findOperators(plan: SparkPlan, predicate: SparkPlan => Boolean): Seq[SparkPlan] = { - def recurse( - plan: SparkPlan, - predicate: SparkPlan => Boolean, - accum: ListBuffer[SparkPlan]): Seq[SparkPlan] = { - if (predicate(plan)) { - accum += plan - } - plan match { - case a: AdaptiveSparkPlanExec => recurse(a.executedPlan, predicate, accum) - case qs: BroadcastQueryStageExec => recurse(qs.broadcast, predicate, accum) - case qs: ShuffleQueryStageExec => recurse(qs.shuffle, predicate, accum) - case other => other.children.flatMap(p => recurse(p, predicate, accum)).headOption - } - accum - } - recurse(plan, predicate, new ListBuffer[SparkPlan]()) - } - - override def skipAssertIsOnTheGpu(plan: SparkPlan): Boolean = false - - override def shouldFailDivOverflow(): Boolean = false - - override def leafNodeDefaultParallelism(ss: SparkSession): Int = { - ss.sparkContext.defaultParallelism - } - - override def shouldFallbackOnAnsiTimestamp(): Boolean = false - - override def getLegacyStatisticalAggregate(): Boolean = true -} diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala index f1c7693f4b7..18bff4c7d27 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala @@ -16,17 +16,497 @@ package com.nvidia.spark.rapids.shims.v2 -import org.apache.spark.sql.internal.SQLConf - -trait Spark31XShims extends Spark30XShims { - override def hasSeparateINT96RebaseConf: Boolean = true - - override def int96ParquetRebaseRead(conf: SQLConf): String = - conf.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ) - override def int96ParquetRebaseWrite(conf: SQLConf): String = - conf.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE) - override def int96ParquetRebaseReadKey: String = - SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ.key - override def int96ParquetRebaseWriteKey: String = - SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE.key +import java.nio.ByteBuffer + +import com.nvidia.spark.InMemoryTableScanMeta +import com.nvidia.spark.rapids._ +import org.apache.arrow.memory.ReferenceManager +import org.apache.arrow.vector.ValueVector + +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging +import org.apache.spark.rapids.shims.v2.GpuShuffleExchangeExec +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.errors.attachTree +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.Average +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeExec} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.python._ +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.rapids._ +import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBase, JoinTypeChecks} +import org.apache.spark.sql.rapids.execution.python._ +import org.apache.spark.sql.rapids.execution.python.shims.v2._ +import org.apache.spark.sql.rapids.shims.v2.{GpuColumnarToRowTransitionExec, HadoopFSUtilsShim} +import org.apache.spark.sql.types._ +import org.apache.spark.storage.{BlockId, BlockManagerId} + +// 31x nondb shims, used by 311cdh and 31x +abstract class Spark31XShims extends Spark301util320Shims with Logging { + + override def getScalaUDFAsExpression( + function: AnyRef, + dataType: DataType, + children: Seq[Expression], + inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil, + outputEncoder: Option[ExpressionEncoder[_]] = None, + udfName: Option[String] = None, + nullable: Boolean = true, + udfDeterministic: Boolean = true): Expression = { + ScalaUDF(function, dataType, children, inputEncoders, outputEncoder, udfName, nullable, + udfDeterministic) + } + + override def getMapSizesByExecutorId( + shuffleId: Int, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, + startMapIndex, endMapIndex, startPartition, endPartition) + } + + override def getFileSourceMaxMetadataValueLength(sqlConf: SQLConf): Int = + sqlConf.maxMetadataStringLength + + override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( + GpuOverrides.expr[Cast]( + "Convert a column of one type of data into another type", + new CastChecks(), + (cast, conf, p, r) => new CastExprMeta[Cast](cast, + SparkSession.active.sessionState.conf.ansiEnabled, conf, p, r, + doFloatToIntCheck = true, stringToAnsiDate = false)), + GpuOverrides.expr[AnsiCast]( + "Convert a column of one type of data into another type", + new CastChecks { + import TypeSig._ + // nullChecks are the same + + override val booleanChecks: TypeSig = integral + fp + BOOLEAN + STRING + DECIMAL_128_FULL + override val sparkBooleanSig: TypeSig = numeric + BOOLEAN + STRING + + override val integralChecks: TypeSig = gpuNumeric + BOOLEAN + STRING + DECIMAL_128_FULL + override val sparkIntegralSig: TypeSig = numeric + BOOLEAN + STRING + + override val fpChecks: TypeSig = (gpuNumeric + BOOLEAN + STRING + DECIMAL_128_FULL) + .withPsNote(TypeEnum.STRING, fpToStringPsNote) + override val sparkFpSig: TypeSig = numeric + BOOLEAN + STRING + + override val dateChecks: TypeSig = TIMESTAMP + DATE + STRING + override val sparkDateSig: TypeSig = TIMESTAMP + DATE + STRING + + override val timestampChecks: TypeSig = TIMESTAMP + DATE + STRING + override val sparkTimestampSig: TypeSig = TIMESTAMP + DATE + STRING + + // stringChecks are the same + // binaryChecks are the same + override val decimalChecks: TypeSig = gpuNumeric + DECIMAL_128_FULL + STRING + override val sparkDecimalSig: TypeSig = numeric + BOOLEAN + STRING + + // calendarChecks are the same + + override val arrayChecks: TypeSig = + ARRAY.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + ARRAY + BINARY + STRUCT) + + psNote(TypeEnum.ARRAY, "The array's child type must also support being cast to " + + "the desired child type") + override val sparkArraySig: TypeSig = ARRAY.nested(all) + + override val mapChecks: TypeSig = + MAP.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + ARRAY + BINARY + STRUCT + MAP) + + psNote(TypeEnum.MAP, "the map's key and value must also support being cast to the " + + "desired child types") + override val sparkMapSig: TypeSig = MAP.nested(all) + + override val structChecks: TypeSig = + STRUCT.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + ARRAY + BINARY + STRUCT) + + psNote(TypeEnum.STRUCT, "the struct's children must also support being cast to the " + + "desired child type(s)") + override val sparkStructSig: TypeSig = STRUCT.nested(all) + + override val udtChecks: TypeSig = none + override val sparkUdtSig: TypeSig = UDT + }, + (cast, conf, p, r) => new CastExprMeta[AnsiCast](cast, ansiEnabled = true, conf = conf, + parent = p, rule = r, doFloatToIntCheck = true, stringToAnsiDate = false)), + GpuOverrides.expr[Average]( + "Average aggregate operator", + ExprChecks.fullAgg( + TypeSig.DOUBLE + TypeSig.DECIMAL_128_FULL, + TypeSig.DOUBLE + TypeSig.DECIMAL_128_FULL, + Seq(ParamCheck("input", + TypeSig.integral + TypeSig.fp + TypeSig.DECIMAL_128_FULL, + TypeSig.numeric))), + (a, conf, p, r) => new AggExprMeta[Average](a, conf, p, r) { + override def tagAggForGpu(): Unit = { + // For Decimal Average the SUM adds a precision of 10 to avoid overflowing + // then it divides by the count with an output scale that is 4 more than the input + // scale. With how our divide works to match Spark, this means that we will need a + // precision of 5 more. So 38 - 10 - 5 = 23 + val dataType = a.child.dataType + dataType match { + case dt: DecimalType => + if (dt.precision > 23) { + if (conf.needDecimalGuarantees) { + willNotWorkOnGpu("GpuAverage cannot guarantee proper overflow checks for " + + s"a precision large than 23. The current precision is ${dt.precision}") + } else { + logWarning("Decimal overflow guarantees disabled for " + + s"Average(${a.child.dataType}) produces ${dt} with an " + + s"intermediate precision of ${dt.precision + 15}") + } + } + case _ => // NOOP + } + GpuOverrides.checkAndTagFloatAgg(dataType, conf, this) + } + + override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = + GpuAverage(childExprs.head) + + // Average is not supported in ANSI mode right now, no matter the type + override val ansiTypeToCheck: Option[DataType] = None + }), + GpuOverrides.expr[Abs]( + "Absolute value", + ExprChecks.unaryProjectAndAstInputMatchesOutput( + TypeSig.implicitCastsAstTypes, TypeSig.gpuNumeric + TypeSig.DECIMAL_128_FULL, + TypeSig.numeric), + (a, conf, p, r) => new UnaryAstExprMeta[Abs](a, conf, p, r) { + // ANSI support for ABS was added in 3.2.0 SPARK-33275 + override def convertToGpu(child: Expression): GpuExpression = GpuAbs(child, false) + }), + GpuOverrides.expr[RegExpReplace]( + "RegExpReplace support for string literal input patterns", + ExprChecks.projectOnly(TypeSig.STRING, TypeSig.STRING, + Seq(ParamCheck("str", TypeSig.STRING, TypeSig.STRING), + ParamCheck("regex", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING), + ParamCheck("rep", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING), + ParamCheck("pos", TypeSig.lit(TypeEnum.INT) + .withPsNote(TypeEnum.INT, "only a value of 1 is supported"), + TypeSig.lit(TypeEnum.INT)))), + (a, conf, p, r) => new GpuRegExpReplaceMeta(a, conf, p, r)).disabledByDefault( + "the implementation is not 100% compatible. " + + "See the compatibility guide for more information."), + // Spark 3.1.1-specific LEAD expression, using custom OffsetWindowFunctionMeta. + GpuOverrides.expr[Lead]( + "Window function that returns N entries ahead of this one", + ExprChecks.windowOnly( + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + TypeSig.NULL + + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all, + Seq( + ParamCheck("input", + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + + TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all), + ParamCheck("offset", TypeSig.INT, TypeSig.INT), + ParamCheck("default", + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + TypeSig.NULL + + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all) + ) + ), + (lead, conf, p, r) => new OffsetWindowFunctionMeta[Lead](lead, conf, p, r) { + override def convertToGpu(): GpuExpression = + GpuLead(input.convertToGpu(), offset.convertToGpu(), default.convertToGpu()) + }), + // Spark 3.1.1-specific LAG expression, using custom OffsetWindowFunctionMeta. + GpuOverrides.expr[Lag]( + "Window function that returns N entries behind this one", + ExprChecks.windowOnly( + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + TypeSig.NULL + + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all, + Seq( + ParamCheck("input", + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + + TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all), + ParamCheck("offset", TypeSig.INT, TypeSig.INT), + ParamCheck("default", + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + TypeSig.NULL + + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all) + ) + ), + (lag, conf, p, r) => new OffsetWindowFunctionMeta[Lag](lag, conf, p, r) { + override def convertToGpu(): GpuExpression = { + GpuLag(input.convertToGpu(), offset.convertToGpu(), default.convertToGpu()) + } + }), + GpuOverrides.expr[GetArrayItem]( + "Gets the field at `ordinal` in the Array", + ExprChecks.binaryProject( + (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + + TypeSig.DECIMAL_128_FULL + TypeSig.MAP).nested(), + TypeSig.all, + ("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + + TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128_FULL + TypeSig.MAP), + TypeSig.ARRAY.nested(TypeSig.all)), + ("ordinal", TypeSig.lit(TypeEnum.INT), TypeSig.INT)), + (in, conf, p, r) => new GpuGetArrayItemMeta(in, conf, p, r){ + override def convertToGpu(arr: Expression, ordinal: Expression): GpuExpression = + GpuGetArrayItem(arr, ordinal, SQLConf.get.ansiEnabled) + }), + GpuOverrides.expr[GetMapValue]( + "Gets Value from a Map based on a key", + ExprChecks.binaryProject(TypeSig.STRING, TypeSig.all, + ("map", TypeSig.MAP.nested(TypeSig.STRING), TypeSig.MAP.nested(TypeSig.all)), + ("key", TypeSig.lit(TypeEnum.STRING), TypeSig.all)), + (in, conf, p, r) => new GpuGetMapValueMeta(in, conf, p, r){ + override def convertToGpu(map: Expression, key: Expression): GpuExpression = + GpuGetMapValue(map, key, SQLConf.get.ansiEnabled) + }), + GpuOverrides.expr[ElementAt]( + "Returns element of array at given(1-based) index in value if column is array. " + + "Returns value for the given key in value if column is map.", + ExprChecks.binaryProject( + (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + + TypeSig.DECIMAL_128_FULL + TypeSig.MAP).nested(), TypeSig.all, + ("array/map", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + + TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128_FULL + TypeSig.MAP) + + TypeSig.MAP.nested(TypeSig.STRING) + .withPsNote(TypeEnum.MAP ,"If it's map, only string is supported."), + TypeSig.ARRAY.nested(TypeSig.all) + TypeSig.MAP.nested(TypeSig.all)), + ("index/key", (TypeSig.lit(TypeEnum.INT) + TypeSig.lit(TypeEnum.STRING)) + .withPsNote(TypeEnum.INT, "ints are only supported as array indexes, " + + "not as maps keys") + .withPsNote(TypeEnum.STRING, "strings are only supported as map keys, " + + "not array indexes"), + TypeSig.all)), + (in, conf, p, r) => new BinaryExprMeta[ElementAt](in, conf, p, r) { + override def tagExprForGpu(): Unit = { + // To distinguish the supported nested type between Array and Map + val checks = in.left.dataType match { + case _: MapType => + // Match exactly with the checks for GetMapValue + ExprChecks.binaryProject(TypeSig.STRING, TypeSig.all, + ("map", TypeSig.MAP.nested(TypeSig.STRING), TypeSig.MAP.nested(TypeSig.all)), + ("key", TypeSig.lit(TypeEnum.STRING), TypeSig.all)) + case _: ArrayType => + // Match exactly with the checks for GetArrayItem + ExprChecks.binaryProject( + (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + + TypeSig.DECIMAL_128_FULL + TypeSig.MAP).nested(), + TypeSig.all, + ("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + + TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128_FULL + TypeSig.MAP), + TypeSig.ARRAY.nested(TypeSig.all)), + ("ordinal", TypeSig.lit(TypeEnum.INT), TypeSig.INT)) + case _ => throw new IllegalStateException("Only Array or Map is supported as input.") + } + checks.tag(this) + } + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { + GpuElementAt(lhs, rhs, SQLConf.get.ansiEnabled) + } + }), + GpuScalaUDFMeta.exprMeta + ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap + + override def getExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = { + Seq( + GpuOverrides.exec[WindowInPandasExec]( + "The backend for Window Aggregation Pandas UDF, Accelerates the data transfer between" + + " the Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled. For now it only supports row based window frame.", + ExecChecks( + (TypeSig.commonCudfTypes + TypeSig.ARRAY).nested(TypeSig.commonCudfTypes), + TypeSig.all), + (winPy, conf, p, r) => new GpuWindowInPandasExecMetaBase(winPy, conf, p, r) { + override val windowExpressions: Seq[BaseExprMeta[NamedExpression]] = + winPy.windowExpression.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + + override def convertToGpu(): GpuExec = { + GpuWindowInPandasExec( + windowExpressions.map(_.convertToGpu()), + partitionSpec.map(_.convertToGpu()), + // leave ordering expression on the CPU, it's not used for GPU computation + winPy.orderSpec, + childPlans.head.convertIfNeeded() + )(winPy.partitionSpec) + } + }).disabledByDefault("it only supports row based frame for now"), + GpuOverrides.exec[FileSourceScanExec]( + "Reading data from files, often from Hive tables", + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + + TypeSig.ARRAY + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all), + (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { + // partition filters and data filters are not run on the GPU + override val childExprs: Seq[ExprMeta[_]] = Seq.empty + + override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this) + + override def convertToGpu(): GpuExec = { + val sparkSession = wrapped.relation.sparkSession + val options = wrapped.relation.options + + val location = replaceWithAlluxioPathIfNeeded( + conf, + wrapped.relation, + wrapped.partitionFilters, + wrapped.dataFilters) + + val newRelation = HadoopFsRelation( + location, + wrapped.relation.partitionSchema, + wrapped.relation.dataSchema, + wrapped.relation.bucketSpec, + GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat), + options)(sparkSession) + + GpuFileSourceScanExec( + newRelation, + wrapped.output, + wrapped.requiredSchema, + wrapped.partitionFilters, + wrapped.optionalBucketSet, + wrapped.optionalNumCoalescedBuckets, + wrapped.dataFilters, + wrapped.tableIdentifier)(conf) + } + }), + GpuOverrides.exec[InMemoryTableScanExec]( + "Implementation of InMemoryTableScanExec to use GPU accelerated Caching", + ExecChecks((TypeSig.commonCudfTypes + TypeSig.DECIMAL_64 + TypeSig.STRUCT + + TypeSig.ARRAY + TypeSig.MAP).nested(), TypeSig.all), + (scan, conf, p, r) => new InMemoryTableScanMeta(scan, conf, p, r)), + GpuOverrides.exec[SortMergeJoinExec]( + "Sort merge join, replacing with shuffled hash join", + JoinTypeChecks.equiJoinExecChecks, + (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), + GpuOverrides.exec[BroadcastHashJoinExec]( + "Implementation of join using broadcast data", + JoinTypeChecks.equiJoinExecChecks, + (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), + GpuOverrides.exec[ShuffledHashJoinExec]( + "Implementation of join using hashed shuffled data", + JoinTypeChecks.equiJoinExecChecks, + (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), + GpuOverrides.exec[ArrowEvalPythonExec]( + "The backend of the Scalar Pandas UDFs. Accelerates the data transfer between the" + + " Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled", + ExecChecks( + (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all), + (e, conf, p, r) => + new SparkPlanMeta[ArrowEvalPythonExec](e, conf, p, r) { + val udfs: Seq[BaseExprMeta[PythonUDF]] = + e.udfs.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val resultAttrs: Seq[BaseExprMeta[Attribute]] = + e.resultAttrs.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + override val childExprs: Seq[BaseExprMeta[_]] = udfs ++ resultAttrs + + override def replaceMessage: String = "partially run on GPU" + override def noReplacementPossibleMessage(reasons: String): String = + s"cannot run even partially on the GPU because $reasons" + + override def convertToGpu(): GpuExec = + GpuArrowEvalPythonExec(udfs.map(_.convertToGpu()).asInstanceOf[Seq[GpuPythonUDF]], + resultAttrs.map(_.convertToGpu()).asInstanceOf[Seq[Attribute]], + childPlans.head.convertIfNeeded(), + e.evalType) + }), + GpuOverrides.exec[MapInPandasExec]( + "The backend for Map Pandas Iterator UDF. Accelerates the data transfer between the" + + " Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled.", + ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all), + (mapPy, conf, p, r) => new GpuMapInPandasExecMeta(mapPy, conf, p, r)), + GpuOverrides.exec[FlatMapGroupsInPandasExec]( + "The backend for Flat Map Groups Pandas UDF, Accelerates the data transfer between the" + + " Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled.", + ExecChecks(TypeSig.commonCudfTypes, TypeSig.all), + (flatPy, conf, p, r) => new GpuFlatMapGroupsInPandasExecMeta(flatPy, conf, p, r)), + GpuOverrides.exec[AggregateInPandasExec]( + "The backend for an Aggregation Pandas UDF, this accelerates the data transfer between" + + " the Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled.", + ExecChecks(TypeSig.commonCudfTypes, TypeSig.all), + (aggPy, conf, p, r) => new GpuAggregateInPandasExecMeta(aggPy, conf, p, r)) + ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap + } + + override def getGpuColumnarToRowTransition(plan: SparkPlan, + exportColumnRdd: Boolean): GpuColumnarToRowExecParent = { + val serName = plan.conf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) + val serClass = ShimLoader.loadClass(serName) + if (serClass == classOf[com.nvidia.spark.ParquetCachedBatchSerializer]) { + GpuColumnarToRowTransitionExec(plan) + } else { + GpuColumnarToRowExec(plan) + } + } + + override def getGpuShuffleExchangeExec( + gpuOutputPartitioning: GpuPartitioning, + child: SparkPlan, + cpuOutputPartitioning: Partitioning, + cpuShuffle: Option[ShuffleExchangeExec]): GpuShuffleExchangeExecBase = { + val shuffleOrigin = cpuShuffle.map(_.shuffleOrigin).getOrElse(ENSURE_REQUIREMENTS) + GpuShuffleExchangeExec(gpuOutputPartitioning, child, shuffleOrigin)(cpuOutputPartitioning) + } + + override def getGpuShuffleExchangeExec( + queryStage: ShuffleQueryStageExec): GpuShuffleExchangeExecBase = { + queryStage.shuffle.asInstanceOf[GpuShuffleExchangeExecBase] + } + + override def sortOrder( + child: Expression, + direction: SortDirection, + nullOrdering: NullOrdering): SortOrder = SortOrder(child, direction, nullOrdering, Seq.empty) + + override def copySortOrderWithNewChild(s: SortOrder, child: Expression) = { + s.copy(child = child) + } + + override def shouldIgnorePath(path: String): Boolean = { + HadoopFSUtilsShim.shouldIgnorePath(path) + } + + override def getLegacyComplexTypeToString(): Boolean = { + SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING) + } + + // Arrow version changed between Spark versions + override def getArrowDataBuf(vec: ValueVector): (ByteBuffer, ReferenceManager) = { + val arrowBuf = vec.getDataBuffer() + (arrowBuf.nioBuffer(), arrowBuf.getReferenceManager) + } + + /** matches SPARK-33008 fix in 3.1.1 */ + override def shouldFailDivByZero(): Boolean = SQLConf.get.ansiEnabled + + /** dropped by SPARK-34234 */ + override def attachTreeIfSupported[TreeType <: TreeNode[_], A]( + tree: TreeType, + msg: String)( + f: => A + ): A = { + attachTree(tree, msg)(f) + } + + override def shouldFallbackOnAnsiTimestamp(): Boolean = SQLConf.get.ansiEnabled + + override def getAdaptiveInputPlan(adaptivePlan: AdaptiveSparkPlanExec): SparkPlan = { + adaptivePlan.inputPlan + } + + override def getLegacyStatisticalAggregate(): Boolean = + SQLConf.get.legacyStatisticalAggregate + + override def hasCastFloatTimestampUpcast: Boolean = false } diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala deleted file mode 100644 index 92e00d04cec..00000000000 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala +++ /dev/null @@ -1,822 +0,0 @@ -/* - * Copyright (c) 2021, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids.shims.v2 - -import java.net.URI -import java.nio.ByteBuffer - -import com.esotericsoftware.kryo.Kryo -import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} -import com.nvidia.spark.InMemoryTableScanMeta -import com.nvidia.spark.rapids._ -import org.apache.arrow.memory.ReferenceManager -import org.apache.arrow.vector.ValueVector -import org.apache.hadoop.fs.{FileStatus, Path} - -import org.apache.spark.SparkEnv -import org.apache.spark.internal.Logging -import org.apache.spark.rapids.shims.v2.GpuShuffleExchangeExec -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.Average -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} -import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.connector.read.Scan -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.command.{AlterTableRecoverPartitionsCommand, RunnableCommand} -import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.rapids.GpuPartitioningUtils -import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan -import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ReusedExchangeExec, ShuffleExchangeExec} -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.python._ -import org.apache.spark.sql.execution.window.WindowExecBase -import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} -import org.apache.spark.sql.rapids._ -import org.apache.spark.sql.rapids.execution.{GpuBroadcastNestedLoopJoinExecBase, GpuShuffleExchangeExecBase, JoinTypeChecks, SerializeBatchDeserializeHostBuffer, SerializeConcatHostBuffersDeserializeBatch} -import org.apache.spark.sql.rapids.execution.python._ -import org.apache.spark.sql.rapids.execution.python.shims.v2._ -import org.apache.spark.sql.rapids.shims.v2.{GpuColumnarToRowTransitionExec, GpuSchemaUtils, HadoopFSUtilsShim} -import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.sql.types._ -import org.apache.spark.storage.{BlockId, BlockManagerId} - -abstract class SparkBaseShims extends Spark31XShims with Logging { - - override def v1RepairTableCommand(tableName: TableIdentifier): RunnableCommand = - AlterTableRecoverPartitionsCommand(tableName) - - override def getScalaUDFAsExpression( - function: AnyRef, - dataType: DataType, - children: Seq[Expression], - inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil, - outputEncoder: Option[ExpressionEncoder[_]] = None, - udfName: Option[String] = None, - nullable: Boolean = true, - udfDeterministic: Boolean = true): Expression = { - ScalaUDF(function, dataType, children, inputEncoders, outputEncoder, udfName, nullable, - udfDeterministic) - } - - override def getMapSizesByExecutorId( - shuffleId: Int, - startMapIndex: Int, - endMapIndex: Int, - startPartition: Int, - endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, - startMapIndex, endMapIndex, startPartition, endPartition) - } - - override def getGpuBroadcastNestedLoopJoinShim( - left: SparkPlan, - right: SparkPlan, - join: BroadcastNestedLoopJoinExec, - joinType: JoinType, - condition: Option[Expression], - targetSizeBytes: Long): GpuBroadcastNestedLoopJoinExecBase = { - GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition, targetSizeBytes) - } - - override def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = { - plan match { - case _: GpuBroadcastHashJoinExec => true - case _ => false - } - } - - override def isWindowFunctionExec(plan: SparkPlan): Boolean = plan.isInstanceOf[WindowExecBase] - - override def isGpuShuffledHashJoin(plan: SparkPlan): Boolean = { - plan match { - case _: GpuShuffledHashJoinExec => true - case _ => false - } - } - - override def getFileSourceMaxMetadataValueLength(sqlConf: SQLConf): Int = - sqlConf.maxMetadataStringLength - - override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( - GpuOverrides.expr[Cast]( - "Convert a column of one type of data into another type", - new CastChecks(), - (cast, conf, p, r) => new CastExprMeta[Cast](cast, - SparkSession.active.sessionState.conf.ansiEnabled, conf, p, r, - doFloatToIntCheck = true, stringToAnsiDate = false)), - GpuOverrides.expr[AnsiCast]( - "Convert a column of one type of data into another type", - new CastChecks { - import TypeSig._ - // nullChecks are the same - - override val booleanChecks: TypeSig = integral + fp + BOOLEAN + STRING + DECIMAL_128_FULL - override val sparkBooleanSig: TypeSig = numeric + BOOLEAN + STRING - - override val integralChecks: TypeSig = gpuNumeric + BOOLEAN + STRING + DECIMAL_128_FULL - override val sparkIntegralSig: TypeSig = numeric + BOOLEAN + STRING - - override val fpChecks: TypeSig = (gpuNumeric + BOOLEAN + STRING + DECIMAL_128_FULL) - .withPsNote(TypeEnum.STRING, fpToStringPsNote) - override val sparkFpSig: TypeSig = numeric + BOOLEAN + STRING - - override val dateChecks: TypeSig = TIMESTAMP + DATE + STRING - override val sparkDateSig: TypeSig = TIMESTAMP + DATE + STRING - - override val timestampChecks: TypeSig = TIMESTAMP + DATE + STRING - override val sparkTimestampSig: TypeSig = TIMESTAMP + DATE + STRING - - // stringChecks are the same - // binaryChecks are the same - override val decimalChecks: TypeSig = gpuNumeric + DECIMAL_128_FULL + STRING - override val sparkDecimalSig: TypeSig = numeric + BOOLEAN + STRING - - // calendarChecks are the same - - override val arrayChecks: TypeSig = - ARRAY.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + ARRAY + BINARY + STRUCT) + - psNote(TypeEnum.ARRAY, "The array's child type must also support being cast to " + - "the desired child type") - override val sparkArraySig: TypeSig = ARRAY.nested(all) - - override val mapChecks: TypeSig = - MAP.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + ARRAY + BINARY + STRUCT + MAP) + - psNote(TypeEnum.MAP, "the map's key and value must also support being cast to the " + - "desired child types") - override val sparkMapSig: TypeSig = MAP.nested(all) - - override val structChecks: TypeSig = - STRUCT.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + ARRAY + BINARY + STRUCT) + - psNote(TypeEnum.STRUCT, "the struct's children must also support being cast to the " + - "desired child type(s)") - override val sparkStructSig: TypeSig = STRUCT.nested(all) - - override val udtChecks: TypeSig = none - override val sparkUdtSig: TypeSig = UDT - }, - (cast, conf, p, r) => new CastExprMeta[AnsiCast](cast, ansiEnabled = true, conf = conf, - parent = p, rule = r, doFloatToIntCheck = true, stringToAnsiDate = false)), - GpuOverrides.expr[Average]( - "Average aggregate operator", - ExprChecks.fullAgg( - TypeSig.DOUBLE + TypeSig.DECIMAL_128_FULL, - TypeSig.DOUBLE + TypeSig.DECIMAL_128_FULL, - Seq(ParamCheck("input", - TypeSig.integral + TypeSig.fp + TypeSig.DECIMAL_128_FULL, - TypeSig.numeric))), - (a, conf, p, r) => new AggExprMeta[Average](a, conf, p, r) { - override def tagAggForGpu(): Unit = { - // For Decimal Average the SUM adds a precision of 10 to avoid overflowing - // then it divides by the count with an output scale that is 4 more than the input - // scale. With how our divide works to match Spark, this means that we will need a - // precision of 5 more. So 38 - 10 - 5 = 23 - val dataType = a.child.dataType - dataType match { - case dt: DecimalType => - if (dt.precision > 23) { - if (conf.needDecimalGuarantees) { - willNotWorkOnGpu("GpuAverage cannot guarantee proper overflow checks for " + - s"a precision large than 23. The current precision is ${dt.precision}") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"Average(${a.child.dataType}) produces ${dt} with an " + - s"intermediate precision of ${dt.precision + 15}") - } - } - case _ => // NOOP - } - GpuOverrides.checkAndTagFloatAgg(dataType, conf, this) - } - - override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = - GpuAverage(childExprs.head) - - // Average is not supported in ANSI mode right now, no matter the type - override val ansiTypeToCheck: Option[DataType] = None - }), - GpuOverrides.expr[Abs]( - "Absolute value", - ExprChecks.unaryProjectAndAstInputMatchesOutput( - TypeSig.implicitCastsAstTypes, TypeSig.gpuNumeric + TypeSig.DECIMAL_128_FULL, - TypeSig.numeric), - (a, conf, p, r) => new UnaryAstExprMeta[Abs](a, conf, p, r) { - // ANSI support for ABS was added in 3.2.0 SPARK-33275 - override def convertToGpu(child: Expression): GpuExpression = GpuAbs(child, false) - }), - GpuOverrides.expr[RegExpReplace]( - "RegExpReplace support for string literal input patterns", - ExprChecks.projectOnly(TypeSig.STRING, TypeSig.STRING, - Seq(ParamCheck("str", TypeSig.STRING, TypeSig.STRING), - ParamCheck("regex", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING), - ParamCheck("rep", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING), - ParamCheck("pos", TypeSig.lit(TypeEnum.INT) - .withPsNote(TypeEnum.INT, "only a value of 1 is supported"), - TypeSig.lit(TypeEnum.INT)))), - (a, conf, p, r) => new GpuRegExpReplaceMeta(a, conf, p, r)).disabledByDefault( - "the implementation is not 100% compatible. " + - "See the compatibility guide for more information."), - // Spark 3.1.1-specific LEAD expression, using custom OffsetWindowFunctionMeta. - GpuOverrides.expr[Lead]( - "Window function that returns N entries ahead of this one", - ExprChecks.windowOnly( - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT).nested(), - TypeSig.all, - Seq( - ParamCheck("input", - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + - TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT).nested(), - TypeSig.all), - ParamCheck("offset", TypeSig.INT, TypeSig.INT), - ParamCheck("default", - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT).nested(), - TypeSig.all) - ) - ), - (lead, conf, p, r) => new OffsetWindowFunctionMeta[Lead](lead, conf, p, r) { - override def convertToGpu(): GpuExpression = - GpuLead(input.convertToGpu(), offset.convertToGpu(), default.convertToGpu()) - }), - // Spark 3.1.1-specific LAG expression, using custom OffsetWindowFunctionMeta. - GpuOverrides.expr[Lag]( - "Window function that returns N entries behind this one", - ExprChecks.windowOnly( - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT).nested(), - TypeSig.all, - Seq( - ParamCheck("input", - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + - TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT).nested(), - TypeSig.all), - ParamCheck("offset", TypeSig.INT, TypeSig.INT), - ParamCheck("default", - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128_FULL + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT).nested(), - TypeSig.all) - ) - ), - (lag, conf, p, r) => new OffsetWindowFunctionMeta[Lag](lag, conf, p, r) { - override def convertToGpu(): GpuExpression = { - GpuLag(input.convertToGpu(), offset.convertToGpu(), default.convertToGpu()) - } - }), - GpuOverrides.expr[GetArrayItem]( - "Gets the field at `ordinal` in the Array", - ExprChecks.binaryProject( - (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128_FULL + TypeSig.MAP).nested(), - TypeSig.all, - ("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128_FULL + TypeSig.MAP), - TypeSig.ARRAY.nested(TypeSig.all)), - ("ordinal", TypeSig.lit(TypeEnum.INT), TypeSig.INT)), - (in, conf, p, r) => new GpuGetArrayItemMeta(in, conf, p, r){ - override def convertToGpu(arr: Expression, ordinal: Expression): GpuExpression = - GpuGetArrayItem(arr, ordinal, SQLConf.get.ansiEnabled) - }), - GpuOverrides.expr[GetMapValue]( - "Gets Value from a Map based on a key", - ExprChecks.binaryProject(TypeSig.STRING, TypeSig.all, - ("map", TypeSig.MAP.nested(TypeSig.STRING), TypeSig.MAP.nested(TypeSig.all)), - ("key", TypeSig.lit(TypeEnum.STRING), TypeSig.all)), - (in, conf, p, r) => new GpuGetMapValueMeta(in, conf, p, r){ - override def convertToGpu(map: Expression, key: Expression): GpuExpression = - GpuGetMapValue(map, key, SQLConf.get.ansiEnabled) - }), - GpuOverrides.expr[ElementAt]( - "Returns element of array at given(1-based) index in value if column is array. " + - "Returns value for the given key in value if column is map.", - ExprChecks.binaryProject( - (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128_FULL + TypeSig.MAP).nested(), TypeSig.all, - ("array/map", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128_FULL + TypeSig.MAP) + - TypeSig.MAP.nested(TypeSig.STRING) - .withPsNote(TypeEnum.MAP ,"If it's map, only string is supported."), - TypeSig.ARRAY.nested(TypeSig.all) + TypeSig.MAP.nested(TypeSig.all)), - ("index/key", (TypeSig.lit(TypeEnum.INT) + TypeSig.lit(TypeEnum.STRING)) - .withPsNote(TypeEnum.INT, "ints are only supported as array indexes, " + - "not as maps keys") - .withPsNote(TypeEnum.STRING, "strings are only supported as map keys, " + - "not array indexes"), - TypeSig.all)), - (in, conf, p, r) => new BinaryExprMeta[ElementAt](in, conf, p, r) { - override def tagExprForGpu(): Unit = { - // To distinguish the supported nested type between Array and Map - val checks = in.left.dataType match { - case _: MapType => - // Match exactly with the checks for GetMapValue - ExprChecks.binaryProject(TypeSig.STRING, TypeSig.all, - ("map", TypeSig.MAP.nested(TypeSig.STRING), TypeSig.MAP.nested(TypeSig.all)), - ("key", TypeSig.lit(TypeEnum.STRING), TypeSig.all)) - case _: ArrayType => - // Match exactly with the checks for GetArrayItem - ExprChecks.binaryProject( - (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128_FULL + TypeSig.MAP).nested(), - TypeSig.all, - ("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128_FULL + TypeSig.MAP), - TypeSig.ARRAY.nested(TypeSig.all)), - ("ordinal", TypeSig.lit(TypeEnum.INT), TypeSig.INT)) - case _ => throw new IllegalStateException("Only Array or Map is supported as input.") - } - checks.tag(this) - } - override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { - GpuElementAt(lhs, rhs, SQLConf.get.ansiEnabled) - } - }), - GpuScalaUDFMeta.exprMeta - ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap - - override def getExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = { - Seq( - GpuOverrides.exec[WindowInPandasExec]( - "The backend for Window Aggregation Pandas UDF, Accelerates the data transfer between" + - " the Java process and the Python process. It also supports scheduling GPU resources" + - " for the Python process when enabled. For now it only supports row based window frame.", - ExecChecks( - (TypeSig.commonCudfTypes + TypeSig.ARRAY).nested(TypeSig.commonCudfTypes), - TypeSig.all), - (winPy, conf, p, r) => new GpuWindowInPandasExecMetaBase(winPy, conf, p, r) { - override val windowExpressions: Seq[BaseExprMeta[NamedExpression]] = - winPy.windowExpression.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - - override def convertToGpu(): GpuExec = { - GpuWindowInPandasExec( - windowExpressions.map(_.convertToGpu()), - partitionSpec.map(_.convertToGpu()), - // leave ordering expression on the CPU, it's not used for GPU computation - winPy.orderSpec, - childPlans.head.convertIfNeeded() - )(winPy.partitionSpec) - } - }).disabledByDefault("it only supports row based frame for now"), - GpuOverrides.exec[FileSourceScanExec]( - "Reading data from files, often from Hive tables", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all), - (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { - // partition filters and data filters are not run on the GPU - override val childExprs: Seq[ExprMeta[_]] = Seq.empty - - override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this) - - override def convertToGpu(): GpuExec = { - val sparkSession = wrapped.relation.sparkSession - val options = wrapped.relation.options - - val location = replaceWithAlluxioPathIfNeeded( - conf, - wrapped.relation, - wrapped.partitionFilters, - wrapped.dataFilters) - - val newRelation = HadoopFsRelation( - location, - wrapped.relation.partitionSchema, - wrapped.relation.dataSchema, - wrapped.relation.bucketSpec, - GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat), - options)(sparkSession) - - GpuFileSourceScanExec( - newRelation, - wrapped.output, - wrapped.requiredSchema, - wrapped.partitionFilters, - wrapped.optionalBucketSet, - wrapped.optionalNumCoalescedBuckets, - wrapped.dataFilters, - wrapped.tableIdentifier)(conf) - } - }), - GpuOverrides.exec[InMemoryTableScanExec]( - "Implementation of InMemoryTableScanExec to use GPU accelerated Caching", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.DECIMAL_64 + TypeSig.STRUCT - + TypeSig.ARRAY + TypeSig.MAP).nested(), TypeSig.all), - (scan, conf, p, r) => new InMemoryTableScanMeta(scan, conf, p, r)), - GpuOverrides.exec[SortMergeJoinExec]( - "Sort merge join, replacing with shuffled hash join", - JoinTypeChecks.equiJoinExecChecks, - (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), - GpuOverrides.exec[BroadcastHashJoinExec]( - "Implementation of join using broadcast data", - JoinTypeChecks.equiJoinExecChecks, - (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), - GpuOverrides.exec[ShuffledHashJoinExec]( - "Implementation of join using hashed shuffled data", - JoinTypeChecks.equiJoinExecChecks, - (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), - GpuOverrides.exec[ArrowEvalPythonExec]( - "The backend of the Scalar Pandas UDFs. Accelerates the data transfer between the" + - " Java process and the Python process. It also supports scheduling GPU resources" + - " for the Python process when enabled", - ExecChecks( - (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), - TypeSig.all), - (e, conf, p, r) => - new SparkPlanMeta[ArrowEvalPythonExec](e, conf, p, r) { - val udfs: Seq[BaseExprMeta[PythonUDF]] = - e.udfs.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val resultAttrs: Seq[BaseExprMeta[Attribute]] = - e.resultAttrs.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[BaseExprMeta[_]] = udfs ++ resultAttrs - - override def replaceMessage: String = "partially run on GPU" - override def noReplacementPossibleMessage(reasons: String): String = - s"cannot run even partially on the GPU because $reasons" - - override def convertToGpu(): GpuExec = - GpuArrowEvalPythonExec(udfs.map(_.convertToGpu()).asInstanceOf[Seq[GpuPythonUDF]], - resultAttrs.map(_.convertToGpu()).asInstanceOf[Seq[Attribute]], - childPlans.head.convertIfNeeded(), - e.evalType) - }), - GpuOverrides.exec[MapInPandasExec]( - "The backend for Map Pandas Iterator UDF. Accelerates the data transfer between the" + - " Java process and the Python process. It also supports scheduling GPU resources" + - " for the Python process when enabled.", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), - TypeSig.all), - (mapPy, conf, p, r) => new GpuMapInPandasExecMeta(mapPy, conf, p, r)), - GpuOverrides.exec[FlatMapGroupsInPandasExec]( - "The backend for Flat Map Groups Pandas UDF, Accelerates the data transfer between the" + - " Java process and the Python process. It also supports scheduling GPU resources" + - " for the Python process when enabled.", - ExecChecks(TypeSig.commonCudfTypes, TypeSig.all), - (flatPy, conf, p, r) => new GpuFlatMapGroupsInPandasExecMeta(flatPy, conf, p, r)), - GpuOverrides.exec[AggregateInPandasExec]( - "The backend for an Aggregation Pandas UDF, this accelerates the data transfer between" + - " the Java process and the Python process. It also supports scheduling GPU resources" + - " for the Python process when enabled.", - ExecChecks(TypeSig.commonCudfTypes, TypeSig.all), - (aggPy, conf, p, r) => new GpuAggregateInPandasExecMeta(aggPy, conf, p, r)) - ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap - } - - override def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = Seq( - GpuOverrides.scan[ParquetScan]( - "Parquet parsing", - (a, conf, p, r) => new ScanMeta[ParquetScan](a, conf, p, r) { - override def tagSelfForGpu(): Unit = GpuParquetScanBase.tagSupport(this) - - override def convertToGpu(): Scan = { - GpuParquetScan(a.sparkSession, - a.hadoopConf, - a.fileIndex, - a.dataSchema, - a.readDataSchema, - a.readPartitionSchema, - a.pushedFilters, - a.options, - a.partitionFilters, - a.dataFilters, - conf) - } - }), - GpuOverrides.scan[OrcScan]( - "ORC parsing", - (a, conf, p, r) => new ScanMeta[OrcScan](a, conf, p, r) { - override def tagSelfForGpu(): Unit = - GpuOrcScanBase.tagSupport(this) - - override def convertToGpu(): Scan = - GpuOrcScan(a.sparkSession, - a.hadoopConf, - a.fileIndex, - a.dataSchema, - a.readDataSchema, - a.readPartitionSchema, - a.options, - a.pushedFilters, - a.partitionFilters, - a.dataFilters, - conf) - }) - ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap - - override def getBuildSide(join: HashJoin): GpuBuildSide = { - GpuJoinUtils.getGpuBuildSide(join.buildSide) - } - - override def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { - GpuJoinUtils.getGpuBuildSide(join.buildSide) - } - - override def getPartitionFileNames( - partitions: Seq[PartitionDirectory]): Seq[String] = { - val files = partitions.flatMap(partition => partition.files) - files.map(_.getPath.getName) - } - - override def getPartitionFileStatusSize(partitions: Seq[PartitionDirectory]): Long = { - partitions.map(_.files.map(_.getLen).sum).sum - } - - override def getPartitionedFiles( - partitions: Array[PartitionDirectory]): Array[PartitionedFile] = { - partitions.flatMap { p => - p.files.map { f => - PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values) - } - } - } - - override def getPartitionSplitFiles( - partitions: Array[PartitionDirectory], - maxSplitBytes: Long, - relation: HadoopFsRelation): Array[PartitionedFile] = { - partitions.flatMap { partition => - partition.files.flatMap { file => - // getPath() is very expensive so we only want to call it once in this block: - val filePath = file.getPath - val isSplitable = relation.fileFormat.isSplitable( - relation.sparkSession, relation.options, filePath) - PartitionedFileUtil.splitFiles( - sparkSession = relation.sparkSession, - file = file, - filePath = filePath, - isSplitable = isSplitable, - maxSplitBytes = maxSplitBytes, - partitionValues = partition.values - ) - } - } - } - - override def getFileScanRDD( - sparkSession: SparkSession, - readFunction: PartitionedFile => Iterator[InternalRow], - filePartitions: Seq[FilePartition]): RDD[InternalRow] = { - new FileScanRDD(sparkSession, readFunction, filePartitions) - } - - override def createFilePartition(index: Int, files: Array[PartitionedFile]): FilePartition = { - FilePartition(index, files) - } - - override def copyBatchScanExec( - batchScanExec: GpuBatchScanExec, - queryUsesInputFile: Boolean): GpuBatchScanExec = { - val scanCopy = batchScanExec.scan match { - case parquetScan: GpuParquetScan => - parquetScan.copy(queryUsesInputFile=queryUsesInputFile) - case orcScan: GpuOrcScan => - orcScan.copy(queryUsesInputFile=queryUsesInputFile) - case _ => throw new RuntimeException("Wrong format") // never reach here - } - batchScanExec.copy(scan=scanCopy) - } - - override def copyFileSourceScanExec( - scanExec: GpuFileSourceScanExec, - queryUsesInputFile: Boolean): GpuFileSourceScanExec = { - scanExec.copy(queryUsesInputFile=queryUsesInputFile)(scanExec.rapidsConf) - } - - override def getGpuColumnarToRowTransition(plan: SparkPlan, - exportColumnRdd: Boolean): GpuColumnarToRowExecParent = { - val serName = plan.conf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) - val serClass = ShimLoader.loadClass(serName) - if (serClass == classOf[com.nvidia.spark.ParquetCachedBatchSerializer]) { - GpuColumnarToRowTransitionExec(plan) - } else { - GpuColumnarToRowExec(plan) - } - } - - override def checkColumnNameDuplication( - schema: StructType, - colType: String, - resolver: Resolver): Unit = { - GpuSchemaUtils.checkColumnNameDuplication(schema, colType, resolver) - } - - override def getGpuShuffleExchangeExec( - gpuOutputPartitioning: GpuPartitioning, - child: SparkPlan, - cpuOutputPartitioning: Partitioning, - cpuShuffle: Option[ShuffleExchangeExec]): GpuShuffleExchangeExecBase = { - val shuffleOrigin = cpuShuffle.map(_.shuffleOrigin).getOrElse(ENSURE_REQUIREMENTS) - GpuShuffleExchangeExec(gpuOutputPartitioning, child, shuffleOrigin)(cpuOutputPartitioning) - } - - override def getGpuShuffleExchangeExec( - queryStage: ShuffleQueryStageExec): GpuShuffleExchangeExecBase = { - queryStage.shuffle.asInstanceOf[GpuShuffleExchangeExecBase] - } - - override def sortOrder( - child: Expression, - direction: SortDirection, - nullOrdering: NullOrdering): SortOrder = SortOrder(child, direction, nullOrdering, Seq.empty) - - override def copySortOrderWithNewChild(s: SortOrder, child: Expression) = { - s.copy(child = child) - } - - override def alias(child: Expression, name: String)( - exprId: ExprId, - qualifier: Seq[String], - explicitMetadata: Option[Metadata]): Alias = { - Alias(child, name)(exprId, qualifier, explicitMetadata) - } - - override def shouldIgnorePath(path: String): Boolean = { - HadoopFSUtilsShim.shouldIgnorePath(path) - } - - override def getLegacyComplexTypeToString(): Boolean = { - SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING) - } - - // Arrow version changed between Spark versions - override def getArrowDataBuf(vec: ValueVector): (ByteBuffer, ReferenceManager) = { - val arrowBuf = vec.getDataBuffer() - (arrowBuf.nioBuffer(), arrowBuf.getReferenceManager) - } - - override def getArrowValidityBuf(vec: ValueVector): (ByteBuffer, ReferenceManager) = { - val arrowBuf = vec.getValidityBuffer - (arrowBuf.nioBuffer(), arrowBuf.getReferenceManager) - } - - override def createTable(table: CatalogTable, - sessionCatalog: SessionCatalog, - tableLocation: Option[URI], - result: BaseRelation) = { - val newTable = table.copy( - storage = table.storage.copy(locationUri = tableLocation), - // We will use the schema of resolved.relation as the schema of the table (instead of - // the schema of df). It is important since the nullability may be changed by the relation - // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). - schema = result.schema) - // Table location is already validated. No need to check it again during table creation. - sessionCatalog.createTable(newTable, ignoreIfExists = false, validateLocation = false) - } - - override def getArrowOffsetsBuf(vec: ValueVector): (ByteBuffer, ReferenceManager) = { - val arrowBuf = vec.getOffsetBuffer - (arrowBuf.nioBuffer(), arrowBuf.getReferenceManager) - } - - /** matches SPARK-33008 fix in 3.1.1 */ - override def shouldFailDivByZero(): Boolean = SQLConf.get.ansiEnabled - - override def replaceWithAlluxioPathIfNeeded( - conf: RapidsConf, - relation: HadoopFsRelation, - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]): FileIndex = { - - val alluxioPathsReplace: Option[Seq[String]] = conf.getAlluxioPathsToReplace - - if (alluxioPathsReplace.isDefined) { - // alluxioPathsReplace: Seq("key->value", "key1->value1") - // turn the rules to the Map with eg - // { s3:/foo -> alluxio://0.1.2.3:19998/foo, - // gs:/bar -> alluxio://0.1.2.3:19998/bar, - // /baz -> alluxio://0.1.2.3:19998/baz } - val replaceMapOption = alluxioPathsReplace.map(rules => { - rules.map(rule => { - val split = rule.split("->") - if (split.size == 2) { - split(0).trim -> split(1).trim - } else { - throw new IllegalArgumentException(s"Invalid setting for " + - s"${RapidsConf.ALLUXIO_PATHS_REPLACE.key}") - } - }).toMap - }) - - replaceMapOption.map(replaceMap => { - - def isDynamicPruningFilter(e: Expression): Boolean = - e.find(_.isInstanceOf[PlanExpression[_]]).isDefined - - val partitionDirs = relation.location.listFiles( - partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) - - // replacement func to check if the file path is prefixed with the string user configured - // if yes, replace it - val replaceFunc = (f: Path) => { - val pathStr = f.toString - val matchedSet = replaceMap.keySet.filter(reg => pathStr.startsWith(reg)) - if (matchedSet.size > 1) { - // never reach here since replaceMap is a Map - throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + - s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule for each " + - s"file path") - } else if (matchedSet.size == 1) { - new Path(pathStr.replaceFirst(matchedSet.head, replaceMap(matchedSet.head))) - } else { - f - } - } - - // replace all of input files - val inputFiles: Seq[Path] = partitionDirs.flatMap(partitionDir => { - replacePartitionDirectoryFiles(partitionDir, replaceFunc) - }) - - // replace all of rootPaths which are already unique - val rootPaths = relation.location.rootPaths.map(replaceFunc) - - val parameters: Map[String, String] = relation.options - - // infer PartitionSpec - val partitionSpec = GpuPartitioningUtils.inferPartitioning( - relation.sparkSession, - rootPaths, - inputFiles, - parameters, - Option(relation.dataSchema), - replaceFunc) - - // generate a new InMemoryFileIndex holding paths with alluxio schema - new InMemoryFileIndex( - relation.sparkSession, - inputFiles, - parameters, - Option(relation.dataSchema), - userSpecifiedPartitionSpec = Some(partitionSpec)) - }).getOrElse(relation.location) - - } else { - relation.location - } - } - - override def replacePartitionDirectoryFiles(partitionDir: PartitionDirectory, - replaceFunc: Path => Path): Seq[Path] = { - partitionDir.files.map(f => replaceFunc(f.getPath)) - } - - override def reusedExchangeExecPfn: PartialFunction[SparkPlan, ReusedExchangeExec] = { - case ShuffleQueryStageExec(_, e: ReusedExchangeExec) => e - case BroadcastQueryStageExec(_, e: ReusedExchangeExec) => e - } - - /** dropped by SPARK-34234 */ - override def attachTreeIfSupported[TreeType <: TreeNode[_], A]( - tree: TreeType, - msg: String)( - f: => A - ): A = { - attachTree(tree, msg)(f) - } - - override def hasAliasQuoteFix: Boolean = false - - override def filesFromFileIndex(fileIndex: PartitioningAwareFileIndex): Seq[FileStatus] = { - fileIndex.allFiles() - } - - override def broadcastModeTransform(mode: BroadcastMode, rows: Array[InternalRow]): Any = - mode.transform(rows) - - override def registerKryoClasses(kryo: Kryo): Unit = { - kryo.register(classOf[SerializeConcatHostBuffersDeserializeBatch], - new KryoJavaSerializer()) - kryo.register(classOf[SerializeBatchDeserializeHostBuffer], - new KryoJavaSerializer()) - } - - override def shouldFallbackOnAnsiTimestamp(): Boolean = SQLConf.get.ansiEnabled - - override def getAdaptiveInputPlan(adaptivePlan: AdaptiveSparkPlanExec): SparkPlan = { - adaptivePlan.inputPlan - } - - override def getLegacyStatisticalAggregate(): Boolean = - SQLConf.get.legacyStatisticalAggregate -} diff --git a/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala similarity index 100% rename from sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala rename to sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/AQEUtils.scala diff --git a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala similarity index 99% rename from sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala rename to sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala index 5fcb23a18c8..e8b4e57457a 100644 --- a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/SparkBaseShims.scala +++ b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala @@ -67,7 +67,7 @@ import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManagerId} -abstract class SparkBaseShims extends Spark30XShims with Logging { +abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { override def v1RepairTableCommand(tableName: TableIdentifier): RunnableCommand = AlterTableRecoverPartitionsCommand(tableName) diff --git a/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShimsBase.scala similarity index 97% rename from sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala rename to sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShimsBase.scala index ce3f2fee778..bfa13b42280 100644 --- a/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala +++ b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShimsBase.scala @@ -31,9 +31,9 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.execution.GpuCustomShuffleReaderExec /** -* Shim base class that can be compiled with every supported 3.0.x +* Shim base class that can be compiled with every supported 31xdb */ -trait Spark30XShims extends SparkShims { +trait Spark31XdbShimsBase extends SparkShims { override def parquetRebaseReadKey: String = SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key override def parquetRebaseWriteKey: String = diff --git a/sql-plugin/src/main/312db/scala/org/apache/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/31xdb/scala/org/apache/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala similarity index 100% rename from sql-plugin/src/main/312db/scala/org/apache/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala rename to sql-plugin/src/main/31xdb/scala/org/apache/spark/rapids/shims/v2/GpuShuffleExchangeExec.scala From 34994dd0d88db66593b4e248deadb04043938a4c Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Fri, 3 Dec 2021 22:56:07 +0800 Subject: [PATCH 19/52] Optimize sample perf (#4159) Signed-off-by: Chong Gao --- docs/configs.md | 1 + .../com/nvidia/spark/rapids/GatherUtils.scala | 45 ++++++ .../com/nvidia/spark/rapids/RapidsConf.scala | 8 + .../spark/rapids/basicPhysicalOperators.scala | 148 +++++++++++++----- .../spark/sql/rapids/GpuPoissonSampler.scala | 41 ++--- 5 files changed, 178 insertions(+), 65 deletions(-) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/GatherUtils.scala diff --git a/docs/configs.md b/docs/configs.md index 06614aa4bb4..3db47bdfbc4 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -79,6 +79,7 @@ Name | Description | Default Value 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 +spark.rapids.sql.fast.sample|Option to turn on fast sample. If enable it is inconsistent with CPU sample because of GPU sample algorithm is inconsistent with CPU.|false spark.rapids.sql.format.csv.enabled|When set to false disables all csv input and output acceleration. (only input is currently supported anyways)|true spark.rapids.sql.format.csv.read.enabled|When set to false disables csv input acceleration|true spark.rapids.sql.format.orc.enabled|When set to false disables all orc input and output acceleration|true diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GatherUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GatherUtils.scala new file mode 100644 index 00000000000..7f117d7729c --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GatherUtils.scala @@ -0,0 +1,45 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.nvidia.spark.rapids + +import scala.collection.mutable.ArrayBuffer + +import ai.rapids.cudf.ColumnVector + +import org.apache.spark.sql.vectorized.ColumnarBatch + +object GatherUtils extends Arm { + def gather(cb: ColumnarBatch, rows: ArrayBuffer[Int]): ColumnarBatch = { + val colTypes = GpuColumnVector.extractTypes(cb) + if (rows.isEmpty) { + GpuColumnVector.emptyBatchFromTypes(colTypes) + } else if (cb.numCols() == 0) { + // for count agg, num of cols is 0 + val c = GpuColumnVector.emptyBatchFromTypes(colTypes) + c.setNumRows(rows.length) + c + } else { + withResource(ColumnVector.fromInts(rows: _*)) { gatherCv => + withResource(GpuColumnVector.from(cb)) { table => + // GPU gather + withResource(table.gather(gatherCv)) { gatheredTable => + GpuColumnVector.from(gatheredTable, colTypes) + } + } + } + } + } +} 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 169840f4b6e..60802477213 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 @@ -1319,6 +1319,12 @@ object RapidsConf { .booleanConf .createWithDefault(value = false) + val ENABLE_FAST_SAMPLE = conf("spark.rapids.sql.fast.sample") + .doc("Option to turn on fast sample. If enable it is inconsistent with CPU sample " + + "because of GPU sample algorithm is inconsistent with CPU.") + .booleanConf + .createWithDefault(value = false) + private def printSectionHeader(category: String): Unit = println(s"\n### $category") @@ -1733,6 +1739,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isCpuBasedUDFEnabled: Boolean = get(ENABLE_CPU_BASED_UDF) + lazy val isFastSampleEnabled: Boolean = get(ENABLE_FAST_SAMPLE) + private val optimizerDefaults = Map( // this is not accurate because CPU projections do have a cost due to appending values // to each row that is produced, but this needs to be a really small number because diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index a882a19dde1..73b719a8bec 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import scala.annotation.tailrec +import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf import ai.rapids.cudf._ @@ -365,19 +366,32 @@ case class GpuFilterExec( } } -class GpuSampleExecMeta(sample: SampleExec, conf: RapidsConf, p: Option[RapidsMeta[_, _, _]], +class GpuSampleExecMeta( + sample: SampleExec, + conf: RapidsConf, + p: Option[RapidsMeta[_, _, _]], r: DataFromReplacementRule) extends SparkPlanMeta[SampleExec](sample, conf, p, r) - with Logging { + with Logging { override def convertToGpu(): GpuExec = { val gpuChild = childPlans.head.convertIfNeeded() - GpuSampleExec(sample.lowerBound, sample.upperBound, sample.withReplacement, - sample.seed, gpuChild) + if (conf.isFastSampleEnabled) { + // Use GPU sample JNI, this is faster, but the output is not the same as CPU produces + GpuFastSampleExec(sample.lowerBound, sample.upperBound, sample.withReplacement, + sample.seed, gpuChild) + } else { + // The output is the same as CPU produces + // First generates row indexes by CPU sampler, then use GPU to gathers + GpuSampleExec(sample.lowerBound, sample.upperBound, sample.withReplacement, + sample.seed, gpuChild) + } } } -case class GpuSampleExec(lowerBound: Double, upperBound: Double, withReplacement: Boolean, - seed: Long, child: SparkPlan) - extends ShimUnaryExecNode with GpuExec { +case class GpuSampleExec( + lowerBound: Double, + upperBound: Double, + withReplacement: Boolean, + seed: Long, child: SparkPlan) extends ShimUnaryExecNode with GpuExec { override lazy val additionalMetrics: Map[String, GpuMetric] = Map( OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME)) @@ -404,7 +418,9 @@ case class GpuSampleExec(lowerBound: Double, upperBound: Double, withReplacement val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) val opTime = gpuLongMetric(OP_TIME) + val rdd = child.executeColumnar() + // CPU consistent, first generates sample row indexes by CPU, then gathers by GPU if (withReplacement) { new GpuPartitionwiseSampledRDD( rdd, @@ -415,46 +431,106 @@ case class GpuSampleExec(lowerBound: Double, upperBound: Double, withReplacement } else { rdd.mapPartitionsWithIndex( (index, iterator) => { - // use CPU sampler generate filter + // use CPU sampler generate row indexes val sampler = new BernoulliCellSampler(lowerBound, upperBound) sampler.setSeed(seed + index) - iterator.map[ColumnarBatch] { batch => - numOutputBatches += 1 - withResource(batch) { b => // will generate new columnar column, close this - val numRows = b.numRows() - val filter = withResource(HostColumnVector.builder(DType.BOOL8, numRows)) { - builder => - (0 until numRows).foreach { _ => - val n = sampler.sample() - if (n > 0) { - builder.append(1.toByte) - numOutputRows += 1 - } else { - builder.append(0.toByte) - } + iterator.map[ColumnarBatch] { columnarBatch => + // collect sampled row idx + // samples idx in batch one by one, so it's same as CPU execution + withResource(new NvtxWithMetrics("Sample Exec", NvtxColor.YELLOW, opTime)) { _ => + withResource(columnarBatch) { cb => + // generate sampled row indexes by CPU + val sampledRows = new ArrayBuffer[Int] + var rowIndex = 0 + while (rowIndex < cb.numRows()) { + if (sampler.sample() > 0) { + sampledRows += rowIndex } - builder.buildAndPutOnDevice() + rowIndex += 1 + } + numOutputBatches += 1 + numOutputRows += sampledRows.length + // gather by row indexes + GatherUtils.gather(cb, sampledRows) } + } + } + } + , preservesPartitioning = true + ) + } + } +} - // use GPU filer rows - val colTypes = GpuColumnVector.extractTypes(b) - withResource(filter) { filter => - withResource(GpuColumnVector.from(b)) { tbl => - withResource(tbl.filter(filter)) { filteredData => - if (filteredData.getRowCount == 0) { - GpuColumnVector.emptyBatchFromTypes(colTypes) - } else { - GpuColumnVector.from(filteredData, colTypes) - } +case class GpuFastSampleExec( + lowerBound: Double, + upperBound: Double, + withReplacement: Boolean, + seed: Long, + child: SparkPlan) extends ShimUnaryExecNode with GpuExec { + + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME)) + + override def output: Seq[Attribute] = { + child.output + } + + // add one coalesce exec to avoid empty batch and small batch, + // because sample will shrink the batch + override val coalesceAfter: Boolean = true + + // Note GPU sample does not preserve the ordering + override def outputOrdering: Seq[SortOrder] = Nil + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def doExecute(): RDD[InternalRow] = + throw new IllegalStateException(s"Row-based execution should not occur for $this") + + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val opTime = gpuLongMetric(OP_TIME) + val rdd = child.executeColumnar() + + // CPU inconsistent, uses GPU sample JNI + rdd.mapPartitionsWithIndex( + (index, iterator) => { + iterator.map[ColumnarBatch] { columnarBatch => + withResource(new NvtxWithMetrics("Fast Sample Exec", NvtxColor.YELLOW, opTime)) { _ => + withResource(columnarBatch) { cb => + numOutputBatches += 1 + val numSampleRows = (cb.numRows() * (upperBound - lowerBound)).toLong + + val colTypes = GpuColumnVector.extractTypes(cb) + if (numSampleRows == 0L) { + GpuColumnVector.emptyBatchFromTypes(colTypes) + } else if (cb.numCols() == 0) { + // for count agg, num of cols is 0 + val c = GpuColumnVector.emptyBatchFromTypes(colTypes) + c.setNumRows(numSampleRows.toInt) + c + } else { + withResource(GpuColumnVector.from(cb)) { table => + // GPU sample + withResource(table.sample(numSampleRows, withReplacement, seed + index)) { + sampled => + val cb = GpuColumnVector.from(sampled, colTypes) + numOutputRows += cb.numRows() + cb } } } } } } - ,preservesPartitioning = true - ) - } + } + , preservesPartitioning = true + ) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuPoissonSampler.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuPoissonSampler.scala index 24d543ac169..24003addfef 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuPoissonSampler.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuPoissonSampler.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.rapids import scala.collection.mutable.ArrayBuffer -import ai.rapids.cudf.{DeviceMemoryBuffer, DType, GatherMap, HostMemoryBuffer, NvtxColor} -import com.nvidia.spark.rapids.{Arm, GpuColumnVector, GpuMetric, NvtxWithMetrics} +import ai.rapids.cudf.NvtxColor +import com.nvidia.spark.rapids.{Arm, GatherUtils, GpuMetric, NvtxWithMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.random.PoissonSampler @@ -37,52 +37,35 @@ class GpuPoissonSampler(fraction: Double, useGapSamplingIfPossible: Boolean, } else { batchIterator.map { columnarBatch => withResource(new NvtxWithMetrics("Sample Exec", NvtxColor.YELLOW, opTime)) { _ => - numOutputBatches += 1 withResource(columnarBatch) { cb => // collect sampled row idx // samples idx in batch one by one, so it's same with CPU version val sampledRows = sample(cb.numRows()) - val intBytes = DType.INT32.getSizeInBytes() - val totalBytes = sampledRows.length * intBytes - withResource(HostMemoryBuffer.allocate(totalBytes)) { hostBuffer => - // copy row idx to host buffer - for (idx <- 0 until sampledRows.length) { - hostBuffer.setInt(idx * intBytes, sampledRows(idx)) - } - - // generate gather map and send to GPU to gather - withResource(DeviceMemoryBuffer.allocate(totalBytes)) { deviceBuffer => - deviceBuffer.copyFromHostBuffer(0, hostBuffer, 0, totalBytes) - withResource(new GatherMap(deviceBuffer).toColumnView(0, sampledRows.length)) { - gatherCv => - val colTypes = GpuColumnVector.extractTypes(cb) - withResource(GpuColumnVector.from(cb)) { table => - withResource(table.gather(gatherCv)) { gatheredTable => - GpuColumnVector.from(gatheredTable, colTypes) - } - } - } - } - } + numOutputBatches += 1 + numOutputRows += sampledRows.length + GatherUtils.gather(cb, sampledRows) } } } } } - // collect the sampled row idx + // collect the sampled row indexes, Note one row can be sampled multiple times private def sample(numRows: Int): ArrayBuffer[Int] = { val buf = new ArrayBuffer[Int] - for (rowIdx <- 0 until numRows) { + var rowIdx = 0 + while (rowIdx < numRows) { // invoke PoissonSampler sample val rowCount = super.sample() if (rowCount > 0) { - numOutputRows += rowCount - for (_ <- 0 until rowCount) { + var i = 0 + while (i < rowCount) { buf += rowIdx + i = i + 1 } } + rowIdx += 1 } buf } From 914a91a4134db8d4a2da1db0d0e363e990902c37 Mon Sep 17 00:00:00 2001 From: Hao Zhu <9665750+viadea@users.noreply.github.com> Date: Sun, 5 Dec 2021 11:03:06 -0800 Subject: [PATCH 20/52] Fix indentation issue in getting-started-k8s guide [skip ci] (#4242) * Fix Getting started on K8s doc indentation issue. Signed-off-by: Hao Zhu * Add some indentation Signed-off-by: Hao Zhu --- .../get-started/getting-started-kubernetes.md | 173 +++++++++--------- 1 file changed, 87 insertions(+), 86 deletions(-) diff --git a/docs/get-started/getting-started-kubernetes.md b/docs/get-started/getting-started-kubernetes.md index d059357d21d..283dff4dc50 100644 --- a/docs/get-started/getting-started-kubernetes.md +++ b/docs/get-started/getting-started-kubernetes.md @@ -271,115 +271,116 @@ Using Spark Operator is another way to submit Spark Applications into a Kubernet 1. Locate the Spark Application jars/files in the docker image when preparing docker image. -For example, assume `/opt/sparkRapidsPlugin/test.py` is inside the docker image. + For example, assume `/opt/sparkRapidsPlugin/test.py` is inside the docker image. -This is because currently only `cluster` deployment mode is supported by Spark Operator. + This is because currently only `cluster` deployment mode is supported by Spark Operator. 2. Create spark operator using `helm`. -Follow [Spark Operator quick start guide](https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/quick-start-guide.md) + Follow [Spark Operator quick start guide](https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/quick-start-guide.md) 3. Create a Spark Application YAML file -For example, create a file named `testpython-rapids.yaml` with the following contents: + For example, create a file named `testpython-rapids.yaml` with the following contents: -``` -apiVersion: "sparkoperator.k8s.io/v1beta2" -kind: SparkApplication -metadata: - name: testpython-rapids - namespace: default -spec: - sparkConf: - "spark.ui.port": "4045" - "spark.rapids.sql.concurrentGpuTasks": "1" - "spark.executor.resource.gpu.amount": "1" - "spark.task.resource.gpu.amount": "1" - "spark.executor.memory": "1g" - "spark.rapids.memory.pinnedPool.size": "2g" - "spark.executor.memoryOverhead": "3g" - "spark.locality.wait": "0s" - "spark.sql.files.maxPartitionBytes": "512m" - "spark.sql.shuffle.partitions": "10" - "spark.plugins": "com.nvidia.spark.SQLPlugin" - "spark.executor.resource.gpu.discoveryScript": "/opt/sparkRapidsPlugin/getGpusResources.sh" - "spark.executor.resource.gpu.vendor": "nvidia.com" - "spark.executor.extraClassPath": "/opt/sparkRapidsPlugin/rapids-4-spark.jar:/opt/sparkRapidsPlugin/cudf.jar" - "spark.driver.extraClassPath": "/opt/sparkRapidsPlugin/rapids-4-spark.jar:/opt/sparkRapidsPlugin/cudf.jar" - type: Python - pythonVersion: 3 - mode: cluster - image: "" - imagePullPolicy: Always - mainApplicationFile: "local:///opt/sparkRapidsPlugin/test.py" - sparkVersion: "3.1.1" - restartPolicy: - type: Never - volumes: - - name: "test-volume" - hostPath: - path: "/tmp" - type: Directory - driver: - cores: 1 - coreLimit: "1200m" - memory: "1024m" - labels: - version: 3.1.1 - serviceAccount: spark - volumeMounts: - - name: "test-volume" - mountPath: "/tmp" - executor: - cores: 1 - instances: 1 - memory: "5000m" - gpu: - name: "nvidia.com/gpu" - quantity: 1 - labels: - version: 3.1.1 - volumeMounts: - - name: "test-volume" - mountPath: "/tmp" -``` + ``` + apiVersion: "sparkoperator.k8s.io/v1beta2" + kind: SparkApplication + metadata: + name: testpython-rapids + namespace: default + spec: + sparkConf: + "spark.ui.port": "4045" + "spark.rapids.sql.concurrentGpuTasks": "1" + "spark.executor.resource.gpu.amount": "1" + "spark.task.resource.gpu.amount": "1" + "spark.executor.memory": "1g" + "spark.rapids.memory.pinnedPool.size": "2g" + "spark.executor.memoryOverhead": "3g" + "spark.locality.wait": "0s" + "spark.sql.files.maxPartitionBytes": "512m" + "spark.sql.shuffle.partitions": "10" + "spark.plugins": "com.nvidia.spark.SQLPlugin" + "spark.executor.resource.gpu.discoveryScript": "/opt/sparkRapidsPlugin/getGpusResources.sh" + "spark.executor.resource.gpu.vendor": "nvidia.com" + "spark.executor.extraClassPath": "/opt/sparkRapidsPlugin/rapids-4-spark.jar:/opt/sparkRapidsPlugin/cudf.jar" + "spark.driver.extraClassPath": "/opt/sparkRapidsPlugin/rapids-4-spark.jar:/opt/sparkRapidsPlugin/cudf.jar" + type: Python + pythonVersion: 3 + mode: cluster + image: "" + imagePullPolicy: Always + mainApplicationFile: "local:///opt/sparkRapidsPlugin/test.py" + sparkVersion: "3.1.1" + restartPolicy: + type: Never + volumes: + - name: "test-volume" + hostPath: + path: "/tmp" + type: Directory + driver: + cores: 1 + coreLimit: "1200m" + memory: "1024m" + labels: + version: 3.1.1 + serviceAccount: spark + volumeMounts: + - name: "test-volume" + mountPath: "/tmp" + executor: + cores: 1 + instances: 1 + memory: "5000m" + gpu: + name: "nvidia.com/gpu" + quantity: 1 + labels: + version: 3.1.1 + volumeMounts: + - name: "test-volume" + mountPath: "/tmp" + ``` 4. Submit the Spark Application -``` -sparkctl create testpython-rapids.yaml -``` + ``` + sparkctl create testpython-rapids.yaml + ``` -Note: `sparkctl` can be built from the Spark Operator repo after [installing golang](https://golang.org/doc/install): -``` -cd sparkctl -go build -o sparkctl -``` + Note: `sparkctl` can be built from the Spark Operator repo after [installing golang](https://golang.org/doc/install): + + ``` + cd sparkctl + go build -o sparkctl + ``` 5. Check the driver log -``` -sparkctl log testpython-rapids -``` + ``` + sparkctl log testpython-rapids + ``` 6. Check the status of this Spark Application -``` -sparkctl status testpython-rapids -``` + ``` + sparkctl status testpython-rapids + ``` 7. Port forwarding when Spark driver is running -``` -sparkctl forward testpython-rapids --local-port 1234 --remote-port 4045 -``` - -Then open browser with `http://localhost:1234/` to check Spark UI. + ``` + sparkctl forward testpython-rapids --local-port 1234 --remote-port 4045 + ``` + + Then open browser with `http://localhost:1234/` to check Spark UI. 8. Delete the Spark Application -``` -sparkctl delete testpython-rapids -``` + ``` + sparkctl delete testpython-rapids + ``` Please refer to [Running Spark on Kubernetes](https://spark.apache.org/docs/latest/running-on-kubernetes.html) for more information. From 2ac5b0efa0fe45bcda980c7ce31a01180a43778b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 5 Dec 2021 17:50:53 -0700 Subject: [PATCH 21/52] Update compatibility guide for known regexp issue (#4287) Signed-off-by: Andy Grove --- docs/compatibility.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index 125256fbe6e..60a3d11b2af 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -504,12 +504,14 @@ Here are some examples of regular expression patterns that are not supported on - Empty groups: `()` - Regular expressions containing null characters (unless the pattern is a simple literal string) - Beginning-of-line and end-of-line anchors (`^` and `$`) are not supported in some contexts, such as when combined -- with a choice (`^|a`) or when used anywhere in `regexp_replace` patterns. + with a choice (`^|a`) or when used anywhere in `regexp_replace` patterns. -In addition to these cases that can be detected, there is also one known issue that can cause incorrect results: +In addition to these cases that can be detected, there are also known issues that can cause incorrect results: - `$` does not match the end of a string if the string ends with a line-terminator ([cuDF issue #9620](https://github.com/rapidsai/cudf/issues/9620)) +- Character classes for negative matches have different behavior between CPU and GPU for multiline + strings. The pattern `[^a]` will match line-terminators on CPU but not on GPU. Work is ongoing to increase the range of regular expressions that can run on the GPU. From 6a54ab635486466fb62afd6b4ba622400ff7aeba Mon Sep 17 00:00:00 2001 From: zhanga5 Date: Mon, 6 Dec 2021 09:00:46 +0800 Subject: [PATCH 22/52] Support code coverage report with single version jar [skip ci] (#4030) * Support code coverage report with single version jar and re-enable code coverage report from premerge job Signed-off-by: Alex Zhang * Extract the udf bytecode to jacoco_classes Extract the class files from rapids-4-spark-udf jar to match the Jacoco.exec report Because they'll be modified in aggregator's shade Signed-off-by: gashen * Apply workaround to override udf class files extracted from dist package for Jacoco report by using rapids-4-spark-udf jar Signed-off-by: Alex Zhang * Remove redundant line from 'spark-premerge-build.sh' Signed-off-by: Alex Zhang Co-authored-by: gashen --- build/coverage-report | 11 ++++++++--- docs/dev/README.md | 7 +++++-- jenkins/Jenkinsfile-blossom.premerge | 12 ++++++++++++ jenkins/spark-premerge-build.sh | 15 +++++++++++++++ 4 files changed, 40 insertions(+), 5 deletions(-) diff --git a/build/coverage-report b/build/coverage-report index a5b8047268d..a4ae19aa84e 100755 --- a/build/coverage-report +++ b/build/coverage-report @@ -23,7 +23,9 @@ TMP_CLASS=${TEMP_CLASS_LOC:-"./target/jacoco_classes/"} HTML_LOC=${HTML_LOCATION:="./target/jacoco-report/"} XML_LOC=${XML_LOCATION:="${HTML_LOC}"} DIST_JAR=${RAPIDS_DIST_JAR:-$(ls ./dist/target/rapids-4-spark_2.12-*.jar | grep -v test | xargs readlink -f)} -SOURCE_DIRS=${SOURCE_DIRS:-"./sql-plugin/src/main/scala/:./sql-plugin/src/main/java/:./shuffle-plugin/src/main/scala/"} +SPK_VER=${JACOCO_SPARK_VER:-"301"} +UDF_JAR=${RAPIDS_UDF_JAR:-$(ls ./udf-compiler/target/spark${SPK_VER}/rapids-4-spark-udf_2.12-*-SNAPSHOT-spark${SPK_VER}.jar | grep -v test | xargs readlink -f)} +SOURCE_DIRS=${SOURCE_DIRS:-"./sql-plugin/src/main/scala/:./sql-plugin/src/main/java/:./shuffle-plugin/src/main/scala/:./udf-compiler/src/main/scala/"} SOURCE_WITH_ARGS="--sourcefiles "$(echo $SOURCE_DIRS | sed -e 's/:/ --sourcefiles /g') @@ -31,8 +33,11 @@ SOURCE_WITH_ARGS="--sourcefiles "$(echo $SOURCE_DIRS | sed -e 's/:/ --sourcefile rm -rf "$TMP_CLASS" mkdir -p "$TMP_CLASS" pushd "$TMP_CLASS" -jar xf "$DIST_JAR" -rm -rf com/nvidia/shaded/ org/openucx/ +jar xf "$DIST_JAR" com org rapids spark3xx-common "spark${SPK_VER}/" +# extract the .class files in udf jar and replace the existing ones in spark3xx-ommon and spark$SPK_VER +# because the class files in udf jar will be modified in aggregator's shade phase +jar xf "$UDF_JAR" com/nvidia/spark/udf +rm -rf com/nvidia/shaded/ org/openucx/ spark3xx-common/com/nvidia/spark/udf/ spark${SPK_VER}/com/nvidia/spark/udf/ popd if [ ! -f "$JDEST" ]; then diff --git a/docs/dev/README.md b/docs/dev/README.md index 453fccdd079..93d6c6199db 100644 --- a/docs/dev/README.md +++ b/docs/dev/README.md @@ -266,8 +266,11 @@ The regular [jacoco maven plugin](https://www.jacoco.org/jacoco/trunk/doc/maven. is not currently [able to support](https://github.com/jacoco/jacoco/issues/965) this type of setup. So if you want to generate a coverage report you need to do it manually. Coverage is collected by default so first run the tests, and then generate the report, this should be run -from the root project directory. It will print out the URL of the report at the end. - +from the root project directory. It will print out the URL of the report at the end. Besides, +coverage report only covers test with Spark 301 by default as [jacoco](https://www.jacoco.org/jacoco/trunk/doc/) +can't support combined jars. If you're testing with different Spark version, please change it +via environment variable `JACOCO_SPARK_VER` before generate coverage report, e.g, `export JACOCO_SPARK_VER=311`. + ```bash mvn clean verify ./build/coverage-report diff --git a/jenkins/Jenkinsfile-blossom.premerge b/jenkins/Jenkinsfile-blossom.premerge index ef7a967f6ea..1d06cf9c270 100644 --- a/jenkins/Jenkinsfile-blossom.premerge +++ b/jenkins/Jenkinsfile-blossom.premerge @@ -315,6 +315,18 @@ pipeline { container('gpu') { timeout(time: 4, unit: 'HOURS') { // step only timeout for test run sh "$PREMERGE_SCRIPT mvn_verify" + step([$class : 'JacocoPublisher', + execPattern : '**/target/jacoco.exec', + classPattern : 'target/jacoco_classes/', + sourceInclusionPattern: '**/*.java,**/*.scala', + sourcePattern : 'shuffle-plugin/src/main/scala/,' + + 'udf-compiler/src/main/scala/,sql-plugin/src/main/java/,' + + 'sql-plugin/src/main/scala/,shims/spark311/src/main/scala/,' + + 'shims/spark301db/src/main/scala/,shims/spark301/src/main/scala/,' + + 'shims/spark302/src/main/scala/,shims/spark303/src/main/scala/,' + + 'shims/spark304/src/main/scala/,shims/spark312/src/main/scala/,' + + 'shims/spark313/src/main/scala/' + ]) } } } diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 2bbd0b071d9..65722cbdd45 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -55,6 +55,21 @@ mvn_verify() { mvn -B $MVN_URM_MIRROR '-Psnapshots,pre-merge' clean verify -Dpytest.TEST_TAGS="premerge_ci_1" \ -Dpytest.TEST_TYPE="pre-commit" -Dpytest.TEST_PARALLEL=4 -Dcuda.version=$CUDA_CLASSIFIER + # The jacoco coverage should have been collected, but because of how the shade plugin + # works and jacoco we need to clean some things up so jacoco will only report for the + # things we care about + SPK_VER=${JACOCO_SPARK_VER:-"301"} + mkdir -p target/jacoco_classes/ + FILE=$(ls dist/target/rapids-4-spark_2.12-*.jar | grep -v test | xargs readlink -f) + UDF_JAR=$(ls ./udf-compiler/target/spark${SPK_VER}/rapids-4-spark-udf_2.12-*-spark${SPK_VER}.jar | grep -v test | xargs readlink -f) + pushd target/jacoco_classes/ + jar xf $FILE com org rapids spark3xx-common "spark${JACOCO_SPARK_VER:-301}/" + # extract the .class files in udf jar and replace the existing ones in spark3xx-ommon and spark$SPK_VER + # because the class files in udf jar will be modified in aggregator's shade phase + jar xf "$UDF_JAR" com/nvidia/spark/udf + rm -rf com/nvidia/shaded/ org/openucx/ spark3xx-common/com/nvidia/spark/udf/ spark${SPK_VER}/com/nvidia/spark/udf/ + popd + # Triggering here until we change the jenkins file rapids_shuffle_smoke_test } From 3e27acc6f91df5ab49e1ade8a1512985ec310b6d Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Sun, 5 Dec 2021 17:21:59 -0800 Subject: [PATCH 23/52] Remove aggregator dependency before deploying dist artifact (#4265) * Use intransitive dependency:get for artifacts Fixes #4253 Signed-off-by: Gera Shegalov * Generate reduced pom Signed-off-by: Gera Shegalov * Update deploy pom file Signed-off-by: Gera Shegalov * Undo -Dtransitive=false * Formatting --- dist/maven-antrun/build-parallel-worlds.xml | 25 +++++++++++++++++++++ dist/pom.xml | 23 ++++++++++++++----- jenkins/deploy.sh | 4 +++- jenkins/version-def.sh | 2 +- 4 files changed, 47 insertions(+), 7 deletions(-) diff --git a/dist/maven-antrun/build-parallel-worlds.xml b/dist/maven-antrun/build-parallel-worlds.xml index bff21cf5faf..66fedb8671f 100644 --- a/dist/maven-antrun/build-parallel-worlds.xml +++ b/dist/maven-antrun/build-parallel-worlds.xml @@ -123,5 +123,30 @@ + + + Generating dependency-reduced-pom.xml + + <dependency> + <groupId>com.nvidia</groupId> + <artifactId>rapids-4-spark-aggregator_\S+?</artifactId> + <version>\S+?</version> + <classifier>\S+?</classifier> + <scope>\S+?</scope> + </dependency> + + + + Generated regex to remove aggregator dependencies: + ${aggregatorDependencyRegex} + + + + + + \ No newline at end of file diff --git a/dist/pom.xml b/dist/pom.xml index c505e9cdb03..0a2606482d9 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -34,11 +34,7 @@ rapids-4-spark-aggregator_${scala.binary.version} ${project.version} ${spark.version.classifier} - - provided + compile + META-INF/maven/** + @@ -299,6 +299,19 @@ + + verify + + run + + reduce-pom-deps-in-the-jar + + + + + + update_config_docs verify diff --git a/jenkins/deploy.sh b/jenkins/deploy.sh index e218c736964..359e1291f12 100755 --- a/jenkins/deploy.sh +++ b/jenkins/deploy.sh @@ -53,6 +53,7 @@ ART_ID=`mvn help:evaluate -q -pl $DIST_PL -Dexpression=project.artifactId -Dforc ART_VER=`mvn help:evaluate -q -pl $DIST_PL -Dexpression=project.version -DforceStdout` FPATH="$DIST_PL/target/$ART_ID-$ART_VER" +POM_FPATH="$DIST_PL/target/extra-resources/META-INF/maven/com.nvidia/$ART_ID/pom.xml" echo "Plan to deploy ${FPATH}.jar to $SERVER_URL (ID:$SERVER_ID)" @@ -83,7 +84,8 @@ $DEPLOY_CMD -Durl=$SERVER_URL -DrepositoryId=$SERVER_ID \ # Distribution jar is a shaded artifact so use the reduced dependency pom. $DEPLOY_CMD -Durl=$SERVER_URL -DrepositoryId=$SERVER_ID \ $SRC_DOC_JARS \ - -Dfile=$FPATH.jar -DgroupId=com.nvidia -DartifactId=$ART_ID -Dversion=$ART_VER -DpomFile=./dist/pom.xml + -Dfile=$FPATH.jar -DgroupId=com.nvidia -DartifactId=$ART_ID -Dversion=$ART_VER \ + -DpomFile="$POM_FPATH" ###### Deploy integration tests jar(s) ###### TESTS_ART_ID=`mvn help:evaluate -q -pl $TESTS_PL -Dexpression=project.artifactId -DforceStdout` diff --git a/jenkins/version-def.sh b/jenkins/version-def.sh index 6e8786eaf6b..975507efe21 100755 --- a/jenkins/version-def.sh +++ b/jenkins/version-def.sh @@ -21,7 +21,7 @@ set -e # $OVERWRITE_PARAMS patten 'abc=123;def=456;' PRE_IFS=$IFS IFS=";" -for VAR in $OVERWRITE_PARAMS;do +for VAR in $OVERWRITE_PARAMS; do echo $VAR && export $VAR done IFS=$PRE_IFS From 7bea7c80362fe54055b7e0cd7ffa672098118f76 Mon Sep 17 00:00:00 2001 From: Peixin Date: Mon, 6 Dec 2021 09:36:49 +0800 Subject: [PATCH 24/52] Update cudfjni version to released 21.12.0 (#4261) Signed-off-by: Peixin Li --- docs/configs.md | 2 +- docs/get-started/getting-started-on-prem.md | 4 ++-- integration_tests/README.md | 2 +- jenkins/version-def.sh | 2 +- pom.xml | 2 +- .../src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala | 2 +- 6 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index 7bf0809f9f1..70d9af409db 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -10,7 +10,7 @@ The following is the list of options that `rapids-plugin-4-spark` supports. On startup use: `--conf [conf key]=[conf value]`. For example: ``` -${SPARK_HOME}/bin/spark --jars 'rapids-4-spark_2.12-21.12.0-SNAPSHOT.jar,cudf-21.12.0-SNAPSHOT-cuda11.jar' \ +${SPARK_HOME}/bin/spark --jars 'rapids-4-spark_2.12-21.12.0-SNAPSHOT.jar,cudf-21.12.0-cuda11.jar' \ --conf spark.plugins=com.nvidia.spark.SQLPlugin \ --conf spark.rapids.sql.incompatibleOps.enabled=true ``` diff --git a/docs/get-started/getting-started-on-prem.md b/docs/get-started/getting-started-on-prem.md index 23301c5d916..4ae8ba5aab4 100644 --- a/docs/get-started/getting-started-on-prem.md +++ b/docs/get-started/getting-started-on-prem.md @@ -55,7 +55,7 @@ CUDA and will not run on other versions. The jars use a maven classifier to keep - CUDA 11.x => classifier cuda11 For example, here is a sample version of the jars and cudf with CUDA 11.0 support: -- cudf-21.12.0-SNAPSHOT-cuda11.jar +- cudf-21.12.0-cuda11.jar - rapids-4-spark_2.12-21.12.0-SNAPSHOT.jar jar that your version of the accelerator depends on. @@ -64,7 +64,7 @@ For simplicity export the location to these jars. This example assumes the sampl been placed in the `/opt/sparkRapidsPlugin` directory: ```shell export SPARK_RAPIDS_DIR=/opt/sparkRapidsPlugin -export SPARK_CUDF_JAR=${SPARK_RAPIDS_DIR}/cudf-21.12.0-SNAPSHOT-cuda11.jar +export SPARK_CUDF_JAR=${SPARK_RAPIDS_DIR}/cudf-21.12.0-cuda11.jar export SPARK_RAPIDS_PLUGIN_JAR=${SPARK_RAPIDS_DIR}/rapids-4-spark_2.12-21.12.0-SNAPSHOT.jar ``` diff --git a/integration_tests/README.md b/integration_tests/README.md index 66a09f98701..a07fb6209a8 100644 --- a/integration_tests/README.md +++ b/integration_tests/README.md @@ -157,7 +157,7 @@ If you just want to verify the SQL replacement is working you will need to add t example assumes CUDA 11.0 is being used. ``` -$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-21.12.0-SNAPSHOT.jar,rapids-4-spark-udf-examples_2.12-21.12.0-SNAPSHOT.jar,cudf-21.12.0-SNAPSHOT-cuda11.jar" ./runtests.py +$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-21.12.0-SNAPSHOT.jar,rapids-4-spark-udf-examples_2.12-21.12.0-SNAPSHOT.jar,cudf-21.12.0-cuda11.jar" ./runtests.py ``` You don't have to enable the plugin for this to work, the test framework will do that for you. diff --git a/jenkins/version-def.sh b/jenkins/version-def.sh index 975507efe21..9d0925c098e 100755 --- a/jenkins/version-def.sh +++ b/jenkins/version-def.sh @@ -26,7 +26,7 @@ for VAR in $OVERWRITE_PARAMS; do done IFS=$PRE_IFS -CUDF_VER=${CUDF_VER:-"21.12.0-SNAPSHOT"} +CUDF_VER=${CUDF_VER:-"21.12.0"} CUDA_CLASSIFIER=${CUDA_CLASSIFIER:-"cuda11"} PROJECT_VER=${PROJECT_VER:-"21.12.0-SNAPSHOT"} PROJECT_TEST_VER=${PROJECT_TEST_VER:-"21.12.0-SNAPSHOT"} diff --git a/pom.xml b/pom.xml index 3efa13b4ce5..d469688de3d 100644 --- a/pom.xml +++ b/pom.xml @@ -739,7 +739,7 @@ ${spark.version} spark${buildver} cuda11 - 21.12.0-SNAPSHOT + 21.12.0 2.12 2.12.15 1.5.10 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 36238dad93d..ee387d88bbe 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 @@ -1357,7 +1357,7 @@ object RapidsConf { |On startup use: `--conf [conf key]=[conf value]`. For example: | |``` - |${SPARK_HOME}/bin/spark --jars 'rapids-4-spark_2.12-21.12.0-SNAPSHOT.jar,cudf-21.12.0-SNAPSHOT-cuda11.jar' \ + |${SPARK_HOME}/bin/spark --jars 'rapids-4-spark_2.12-21.12.0-SNAPSHOT.jar,cudf-21.12.0-cuda11.jar' \ |--conf spark.plugins=com.nvidia.spark.SQLPlugin \ |--conf spark.rapids.sql.incompatibleOps.enabled=true |``` From b7e5467b2ed532ddf665ca8cce91c70ff038ac58 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 6 Dec 2021 12:08:24 -0600 Subject: [PATCH 25/52] Revert commonizing the int96ParquetRebase* functions which are actually (#4306) different between Spark 3.0.x and 3.1.x Signed-off-by: Thomas Graves --- .../nvidia/spark/rapids/shims/v2/Spark30XShims.scala | 10 ++++++++++ .../spark/rapids/shims/v2/Spark301util320Shims.scala | 9 --------- .../nvidia/spark/rapids/shims/v2/Spark31XShims.scala | 10 ++++++++++ 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala index a40fd16ff1b..18680d3b902 100644 --- a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala +++ b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala @@ -48,6 +48,16 @@ import org.apache.spark.storage.{BlockId, BlockManagerId} import org.apache.spark.unsafe.types.CalendarInterval abstract class Spark30XShims extends Spark301util320Shims with Logging { + override def int96ParquetRebaseRead(conf: SQLConf): String = + parquetRebaseRead(conf) + override def int96ParquetRebaseWrite(conf: SQLConf): String = + parquetRebaseWrite(conf) + override def int96ParquetRebaseReadKey: String = + parquetRebaseReadKey + override def int96ParquetRebaseWriteKey: String = + parquetRebaseWriteKey + override def hasSeparateINT96RebaseConf: Boolean = false + override def getScalaUDFAsExpression( function: AnyRef, dataType: DataType, diff --git a/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala b/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala index 457dcc650c0..c67af1d2972 100644 --- a/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala +++ b/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala @@ -72,15 +72,6 @@ trait Spark301util320Shims extends SparkShims { conf.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ) override def parquetRebaseWrite(conf: SQLConf): String = conf.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE) - override def int96ParquetRebaseRead(conf: SQLConf): String = - parquetRebaseRead(conf) - override def int96ParquetRebaseWrite(conf: SQLConf): String = - parquetRebaseWrite(conf) - override def int96ParquetRebaseReadKey: String = - parquetRebaseReadKey - override def int96ParquetRebaseWriteKey: String = - parquetRebaseWriteKey - override def hasSeparateINT96RebaseConf: Boolean = false override def sessionFromPlan(plan: SparkPlan): SparkSession = { plan.sqlContext.sparkSession diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala index 18bff4c7d27..015730379ab 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala @@ -52,6 +52,16 @@ import org.apache.spark.storage.{BlockId, BlockManagerId} // 31x nondb shims, used by 311cdh and 31x abstract class Spark31XShims extends Spark301util320Shims with Logging { + override def int96ParquetRebaseRead(conf: SQLConf): String = + conf.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ) + override def int96ParquetRebaseWrite(conf: SQLConf): String = + conf.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE) + override def int96ParquetRebaseReadKey: String = + SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ.key + override def int96ParquetRebaseWriteKey: String = + SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE.key + override def hasSeparateINT96RebaseConf: Boolean = true + override def getScalaUDFAsExpression( function: AnyRef, dataType: DataType, From d3c5847be52e29ea22bce34b77ee2406e0fbbbde Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 6 Dec 2021 11:29:41 -0700 Subject: [PATCH 26/52] Make regexp pattern `[^a]` consistent with Spark for multiline strings (#4255) * Make regular expression pattern `[^a]` consistent with Spark with resepect to newline characters Signed-off-by: Andy Grove * update test name * Handle edge cases * code cleanup --- .../src/main/python/string_test.py | 15 ++++++ .../com/nvidia/spark/rapids/RegexParser.scala | 51 ++++++++++++++++--- .../RegularExpressionTranspilerSuite.scala | 9 +++- 3 files changed, 68 insertions(+), 7 deletions(-) diff --git a/integration_tests/src/main/python/string_test.py b/integration_tests/src/main/python/string_test.py index f4f42924878..ffbad689653 100644 --- a/integration_tests/src/main/python/string_test.py +++ b/integration_tests/src/main/python/string_test.py @@ -523,6 +523,21 @@ def test_regexp_replace_null_pattern_fallback(): 'RegExpReplace', conf={'spark.rapids.sql.expression.RegExpReplace': 'true'}) +def test_regexp_replace_character_set_negated(): + gen = mk_str_gen('[abcd]{0,3}[\r\n]{0,2}[abcd]{0,3}') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp_replace(a, "([^a])|([^b])", "1")', + 'regexp_replace(a, "[^a]", "1")', + 'regexp_replace(a, "([^a]|[\r\n])", "1")', + 'regexp_replace(a, "[^a\r\n]", "1")', + 'regexp_replace(a, "[^a\r]", "1")', + 'regexp_replace(a, "[^a\n]", "1")', + 'regexp_replace(a, "[^\r\n]", "1")', + 'regexp_replace(a, "[^\r]", "1")', + 'regexp_replace(a, "[^\n]", "1")'), + conf={'spark.rapids.sql.expression.RegExpReplace': 'true'}) + def test_rlike(): gen = mk_str_gen('[abcd]{1,3}') assert_gpu_and_cpu_are_equal_collect( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index a2d4d347fb1..43ac51cb56d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -169,11 +169,9 @@ class RegexParser(pattern: String) { throw new RegexUnsupportedException( s"unexpected EOF while parsing escaped character", Some(pos)) case Some(ch) => - ch match { - case '\\' | '^' | '-' | ']' | '+' => - // escaped metacharacter within character class - characterClass.appendEscaped(consumeExpected(ch)) - } + // typically an escaped metacharacter ('\\', '^', '-', ']', '+') + // within the character class, but could be any escaped character + characterClass.appendEscaped(consumeExpected(ch)) } case '\u0000' => throw new RegexUnsupportedException( @@ -494,7 +492,48 @@ class CudfRegexTranspiler(replace: Boolean) { } val components: Seq[RegexCharacterClassComponent] = characters .map(x => rewrite(x).asInstanceOf[RegexCharacterClassComponent]) - RegexCharacterClass(negated, ListBuffer(components: _*)) + + if (negated) { + // There are differences between cuDF and Java handling of newlines + // for negative character matches. The expression `[^a]` will match + // `\r` and `\n` in Java but not in cuDF, so we replace `[^a]` with + // `(?:[\r\n]|[^a])`. We also have to take into account whether any + // newline characters are included in the character range. + // + // Examples: + // + // `[^a]` => `(?:[\r\n]|[^a])` + // `[^a\r]` => `(?:[\n]|[^a])` + // `[^a\n]` => `(?:[\r]|[^a])` + // `[^a\r\n]` => `[^a]` + // `[^\r\n]` => `[^\r\n]` + + val linefeedCharsInPattern = components.flatMap { + case RegexChar(ch) if ch == '\n' || ch == '\r' => Seq(ch) + case RegexEscaped(ch) if ch == 'n' => Seq('\n') + case RegexEscaped(ch) if ch == 'r' => Seq('\r') + case _ => Seq.empty + } + + val onlyLinefeedChars = components.length == linefeedCharsInPattern.length + + val negatedNewlines = Seq('\r', '\n').diff(linefeedCharsInPattern.distinct) + + if (onlyLinefeedChars && linefeedCharsInPattern.length == 2) { + // special case for `[^\r\n]` and `[^\\r\\n]` + RegexCharacterClass(negated = true, ListBuffer(components: _*)) + } else if (negatedNewlines.isEmpty) { + RegexCharacterClass(negated = true, ListBuffer(components: _*)) + } else { + RegexGroup(capture = false, + RegexChoice( + RegexCharacterClass(negated = false, + characters = ListBuffer(negatedNewlines.map(RegexChar): _*)), + RegexCharacterClass(negated = true, ListBuffer(components: _*)))) + } + } else { + RegexCharacterClass(negated, ListBuffer(components: _*)) + } case RegexSequence(parts) => if (parts.isEmpty) { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala index 240ba7d6eab..73295d02ecb 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala @@ -165,7 +165,7 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { test("transpile character class unescaped range symbol") { val patterns = Seq("a[-b]", "a[+-]", "a[-+]", "a[-]", "a[^-]") - val expected = Seq(raw"a[\-b]", raw"a[+\-]", raw"a[\-+]", raw"a[\-]", raw"a[^\-]") + val expected = Seq(raw"a[\-b]", raw"a[+\-]", raw"a[\-+]", raw"a[\-]", "a(?:[\r\n]|[^\\-])") val transpiler = new CudfRegexTranspiler(replace=false) val transpiled = patterns.map(transpiler.transpile) assert(transpiled === expected) @@ -248,6 +248,13 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { assertCpuGpuMatchesRegexpReplace(patterns, inputs) } + test("compare CPU and GPU: regexp replace negated character class") { + val inputs = Seq("a", "b", "a\nb", "a\r\nb\n\rc\rd") + val patterns = Seq("[^z]", "[^\r]", "[^\n]", "[^\r]", "[^\r\n]", + "[^a\n]", "[^b\r]", "[^bc\r\n]", "[^\\r\\n]") + assertCpuGpuMatchesRegexpReplace(patterns, inputs) + } + test("compare CPU and GPU: regexp replace fuzz test with limited chars") { // testing with this limited set of characters finds issues much // faster than using the full ASCII set From 3fe429c0c96ce0e95fbedd7a78aef9002d3a5c95 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 6 Dec 2021 14:55:19 -0600 Subject: [PATCH 27/52] Full support for SUM overflow detection on decimal [databricks] (#4272) Signed-off-by: Robert (Bobby) Evans Co-authored-by: Jason Lowe --- docs/compatibility.md | 83 ++-- docs/supported_ops.md | 8 +- .../src/main/python/hash_aggregate_test.py | 85 +++- .../src/main/python/window_function_test.py | 8 +- .../spark/sql/rapids/aggregate/GpuSum.scala | 11 +- .../spark/sql/rapids/aggregate/GpuSum.scala | 13 +- .../com/nvidia/spark/rapids/GpuCast.scala | 30 +- .../nvidia/spark/rapids/GpuOverrides.scala | 18 +- .../nvidia/spark/rapids/GpuWindowExec.scala | 8 +- .../spark/rapids/GpuWindowExpression.scala | 147 ++++++- .../com/nvidia/spark/rapids/aggregate.scala | 90 +--- .../spark/sql/rapids/AggregateFunctions.scala | 400 ++++++++++++++++-- .../apache/spark/sql/rapids/arithmetic.scala | 43 +- 13 files changed, 669 insertions(+), 275 deletions(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index abed1dd0f56..7b4261f7df6 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -109,63 +109,31 @@ a few operations that we cannot support to the same degree as Spark can on the C ### Decimal Sum Aggregation -When Apache Spark does a sum aggregation on decimal values it will store the result in a value -with a precision that is the input precision + 10, but with a maximum precision of 38. The table -below shows the number of rows/values in an aggregation before an overflow is possible, -and the number of rows/values in the aggregation before an overflow might not be detected. -The numbers are for Spark 3.1.0 and above after a number of fixes were put in place, please see +A number of fixes for overflow detection went into Spark 3.1.0. Please see [SPARK-28067](https://issues.apache.org/jira/browse/SPARK-28067) and -[SPARK-32018](https://issues.apache.org/jira/browse/SPARK-32018) for more information. -Please also note that these are for the worst case situations, meaning all the values in the sum -were either the largest or smallest values possible to be stored in the input type. In the common -case, where the numbers are smaller, or vary between positive and negative values, many more -rows/values can be processed without any issues. - -|Input Precision|Number of values before overflow is possible|Maximum number of values for guaranteed overflow detection (Spark CPU)|Maximum number of values for guaranteed overflow detection (RAPIDS GPU)| -|---------------|------------------------------|------------|-------------| -|1 |11,111,111,111 |2,049,638,219,301,061,290 |Same as CPU | -|2 |10,101,010,101 |186,330,738,118,278,299 |Same as CPU | -|3 |10,010,010,010 |18,465,199,272,982,534 |Same as CPU | -|4 |10,001,000,100 |1,844,848,892,260,181 |Same as CPU | -|5 |10,000,100,001 |184,459,285,329,948 |Same as CPU | -|6 |10,000,010,000 |18,436,762,510,472 |Same as CPU | -|7 |10,000,001,000 |1,834,674,590,838 |Same as CPU | -|8 |10,000,000,100 |174,467,442,481 |Same as CPU | -|9 |10,000,000,010 |Unlimited |Unlimited | -|10 - 19 |10,000,000,000 |Unlimited |Unlimited | -|20 |10,000,000,000 |Unlimited |3,402,823,659,209,384,634 | -|21 |10,000,000,000 |Unlimited |340,282,356,920,938,463 | -|22 |10,000,000,000 |Unlimited |34,028,226,692,093,846 | -|23 |10,000,000,000 |Unlimited |3,402,813,669,209,384 | -|24 |10,000,000,000 |Unlimited |340,272,366,920,938 | -|25 |10,000,000,000 |Unlimited |34,018,236,692,093 | -|26 |10,000,000,000 |Unlimited |3,392,823,669,209 | -|27 |10,000,000,000 |Unlimited |330,282,366,920 | -|28 |10,000,000,000 |Unlimited |24,028,236,692 | -|29 |1,000,000,000 |Unlimited |Falls back to CPU | -|30 |100,000,000 |Unlimited |Falls back to CPU | -|31 |10,000,000 |Unlimited |Falls back to CPU | -|32 |1,000,000 |Unlimited |Falls back to CPU | -|33 |100,000 |Unlimited |Falls back to CPU | -|34 |10,00 |Unlimited |Falls back to CPU | -|35 |1,000 |Unlimited |Falls back to CPU | -|36 |100 |Unlimited |Falls back to CPU | -|37 |10 |Unlimited |Falls back to CPU | -|38 |1 |Unlimited |Falls back to CPU | - -For an input precision of 9 and above, Spark will do the aggregations as a `BigDecimal` -value which is slow, but guarantees that any overflow can be detected. For inputs with a -precision of 8 or below Spark will internally do the calculations as a long value, 64-bits. -When the precision is 8, you would need at least 174-billion values/rows contributing to a -single aggregation result, and even then all the values would need to be either the largest -or the smallest value possible to be stored in the type before the overflow is no longer detected. - -For the RAPIDS Accelerator we only have access to at most a 128-bit value to store the results -in and still detect overflow. Because of this we cannot guarantee overflow detection in all -cases. In some cases we can guarantee unlimited overflow detection because of the maximum number of -values that RAPIDS will aggregate in a single batch. But even in the worst cast for a decimal value -with a precision of 28 the user would still have to aggregate so many values that it overflows 2.4 -times over before we are no longer able to detect it. +[SPARK-32018](https://issues.apache.org/jira/browse/SPARK-32018) for more detailed information. +Some of these fixes we were able to back port, but some of them require Spark 3.1.0 or above to +fully be able to detect overflow in all cases. As such on versions of Spark older than 3.1.0 for +large decimal values there is the possibility of data corruption in some corner cases. +This is true for both the CPU and GPU implementations, but there are fewer of these cases for the +GPU. If this concerns you, you should upgrade to Spark 3.1.0 or above. + +When Apache Spark does a sum aggregation on decimal values it will store the result in a value +with a precision that is the input precision + 10, but with a maximum precision of 38. +For an input precision of 9 and above, Spark will do the aggregations as a Java `BigDecimal` +value which is slow, but guarantees that any overflow can be detected because it can work with +effectively unlimited precision. For inputs with a precision of 8 or below Spark will internally do +the calculations as a long value, 64-bits. When the precision is 8, you would need at least +174,467,442,482 values/rows contributing to a single aggregation result before the overflow is no +longer detected. Even then all the values would need to be either the largest or the smallest value +possible to be stored in the type for the overflow to cause data corruption. + +For the RAPIDS Accelerator we don't have direct access to unlimited precision for our calculations +like the CPU does. For input values with a precision of 8 and below we follow Spark and process the +data the same way, as a 64-bit value. For larger values we will do extra calculations looking at the +higher order digits to be able to detect overflow in all cases. But because of this you may see +some performance differences depending on the input precision used. The differences will show up +when going from an input precision of 8 to 9 and again when going from an input precision of 28 to 29. ### Decimal Average @@ -175,8 +143,7 @@ have. It also inherits some issues from Spark itself. See https://issues.apache.org/jira/browse/SPARK-37024 for a detailed description of some issues with average in Spark. -In order to be able to guarantee overflow detection on the sum with at least 100-billion values -and to be able to guarantee doing the divide with half up rounding at the end we only support +In order to be able to guarantee doing the divide with half up rounding at the end we only support average on input values with a precision of 23 or below. This is 38 - 10 for the sum guarantees and then 5 less to be able to shift the left-hand side of the divide enough to get a correct answer that can be rounded to the result that Spark would produce. diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 1aac3b00cc2..051c9a1906d 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -386,13 +386,13 @@ Accelerator supports are described below. S PS
UTC is only supported TZ for TIMESTAMP
S -PS
max DECIMAL precision of 18
+S S NS NS -PS
max child DECIMAL precision of 18;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
max child DECIMAL precision of 18;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
max child DECIMAL precision of 18;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
NS diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 90d062fc816..406ec0fb694 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -265,33 +265,64 @@ def get_params(init_list, marked_params=[]): ('b', decimal_gen_20_2), ('c', decimal_gen_20_2)] +# NOTE on older versions of Spark decimal 38 causes the CPU to crash +# instead of detect overflows, we have versions of this for both +# 36 and 38 so we can get some coverage for old versions and full +# coverage for newer versions +_grpkey_short_very_big_decimals = [ + ('a', RepeatSeqGen(short_gen, length=50)), + ('b', decimal_gen_36_5), + ('c', decimal_gen_36_5)] + +_grpkey_short_very_big_neg_scale_decimals = [ + ('a', RepeatSeqGen(short_gen, length=50)), + ('b', decimal_gen_36_neg5), + ('c', decimal_gen_36_neg5)] + +_grpkey_short_full_decimals = [ + ('a', RepeatSeqGen(short_gen, length=50)), + ('b', decimal_gen_38_0), + ('c', decimal_gen_38_0)] + +_grpkey_short_full_neg_scale_decimals = [ + ('a', RepeatSeqGen(short_gen, length=50)), + ('b', decimal_gen_38_neg10), + ('c', decimal_gen_38_neg10)] + + _init_list_no_nans_with_decimal = _init_list_no_nans + [ _grpkey_small_decimals] _init_list_no_nans_with_decimalbig = _init_list_no_nans + [ - _grpkey_small_decimals, _grpkey_short_mid_decimals, _grpkey_short_big_decimals] + _grpkey_small_decimals, _grpkey_short_mid_decimals, + _grpkey_short_big_decimals, _grpkey_short_very_big_decimals, + _grpkey_short_very_big_neg_scale_decimals] + +_init_list_full_decimal = [_grpkey_short_full_decimals, + _grpkey_short_full_neg_scale_decimals] -#TODO when we can support sum on larger types https://github.com/NVIDIA/spark-rapids/issues/3944 -# we should move to a larger type and use a smaller count so we can avoid the long CPU run -# then we should look at splitting the reduction up so we do half on the CPU and half on the GPU -# So we can test compatabiliy too (but without spending even longer computing) -def test_hash_reduction_decimal_overflow_sum(): +#Any smaller precision takes way too long to process on the CPU +@pytest.mark.parametrize('precision', [38, 37, 36, 35, 34, 33, 32, 31, 30], ids=idfn) +def test_hash_reduction_decimal_overflow_sum(precision): + constant = '9' * precision + count = pow(10, 38 - precision) assert_gpu_and_cpu_are_equal_collect( - # Spark adds +10 to precision so we need 10-billion entries before overflow is even possible. - # we use 10-billion and 2 to make sure we hit the overflow. - lambda spark: spark.range(0, 10000000002, 1, 48)\ - .selectExpr("CAST('9999999999' as Decimal(10, 0)) as a")\ + lambda spark: spark.range(count)\ + .selectExpr("CAST('{}' as Decimal({}, 0)) as a".format(constant, precision))\ .selectExpr("SUM(a)"), - # set the batch size small because we can have limited GPU memory and the first select - # doubles the size of the batch - conf = {'spark.rapids.sql.batchSizeBytes': '64m'}) + # This is set to 128m because of a number of other bugs that compound to having us + # run out of memory in some setups. These should not happen in production, because + # we really are just doing a really bad job at multiplying to get this result so + # some optimizations are conspiring against us. + conf = {'spark.rapids.sql.batchSizeBytes': '128m'}) @pytest.mark.parametrize('data_gen', [_longs_with_nulls], ids=idfn) def test_hash_grpby_sum_count_action(data_gen): assert_gpu_and_cpu_row_counts_equal( lambda spark: gen_df(spark, data_gen, length=100).groupby('a').agg(f.sum('b')) ) + @pytest.mark.parametrize('data_gen', [_longs_with_nulls], ids=idfn) def test_hash_reduction_sum_count_action(data_gen): assert_gpu_and_cpu_row_counts_equal( @@ -307,19 +338,41 @@ def test_hash_reduction_sum_count_action(data_gen): def test_hash_grpby_sum(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100).groupby('a').agg(f.sum('b')), - conf=conf - ) + conf = copy_and_update(allow_negative_scale_of_decimal_conf, conf)) +@pytest.mark.skipif(is_before_spark_311(), reason="SUM overflows for CPU were fixed in Spark 3.1.1") +@shuffle_test @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', numeric_gens + decimal_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list_full_decimal, ids=idfn) +@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) +def test_hash_grpby_sum_full_decimal(data_gen, conf): + assert_gpu_and_cpu_are_equal_collect( + lambda spark: gen_df(spark, data_gen, length=100).groupby('a').agg(f.sum('b')), + conf = copy_and_update(allow_negative_scale_of_decimal_conf, conf)) + +@approximate_float +@ignore_order +@incompat +@pytest.mark.parametrize('data_gen', numeric_gens + decimal_gens + [decimal_gen_20_2, decimal_gen_30_2, decimal_gen_36_5, decimal_gen_36_neg5], ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) def test_hash_reduction_sum(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen, length=100).selectExpr("SUM(a)"), conf = copy_and_update(allow_negative_scale_of_decimal_conf, conf)) +@pytest.mark.skipif(is_before_spark_311(), reason="SUM overflows for CPU were fixed in Spark 3.1.1") +@approximate_float +@ignore_order +@incompat +@pytest.mark.parametrize('data_gen', numeric_gens + decimal_gens + [decimal_gen_38_0, decimal_gen_38_10, decimal_gen_38_neg10], ids=idfn) +@pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) +def test_hash_reduction_sum_full_decimal(data_gen, conf): + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, data_gen, length=100).selectExpr("SUM(a)"), + conf = copy_and_update(allow_negative_scale_of_decimal_conf, conf)) + @approximate_float @ignore_order @incompat diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 97c045ed5e5..0bf6f1f0bc5 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -164,7 +164,7 @@ def test_decimal128_count_window_no_part(data_gen): conf = allow_negative_scale_of_decimal_conf) @ignore_order -@pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', decimal_gens + decimal_128_gens, ids=idfn) def test_decimal_sum_window(data_gen): assert_gpu_and_cpu_are_equal_sql( lambda spark: three_col_df(spark, byte_gen, LongRangeGen(), data_gen), @@ -177,7 +177,7 @@ def test_decimal_sum_window(data_gen): conf = allow_negative_scale_of_decimal_conf) @ignore_order -@pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', decimal_gens + decimal_128_gens, ids=idfn) def test_decimal_sum_window_no_part(data_gen): assert_gpu_and_cpu_are_equal_sql( lambda spark: two_col_df(spark, LongRangeGen(), data_gen), @@ -191,7 +191,7 @@ def test_decimal_sum_window_no_part(data_gen): @ignore_order -@pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', decimal_gens + decimal_128_gens, ids=idfn) def test_decimal_running_sum_window(data_gen): assert_gpu_and_cpu_are_equal_sql( lambda spark: three_col_df(spark, byte_gen, LongRangeGen(), data_gen), @@ -205,7 +205,7 @@ def test_decimal_running_sum_window(data_gen): {'spark.rapids.sql.batchSizeBytes': '100'})) @ignore_order -@pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', decimal_gens + decimal_128_gens, ids=idfn) def test_decimal_running_sum_window_no_part(data_gen): assert_gpu_and_cpu_are_equal_sql( lambda spark: two_col_df(spark, LongRangeGen(), data_gen), diff --git a/sql-plugin/src/main/301until310-all/scala/org/apache/spark/sql/rapids/aggregate/GpuSum.scala b/sql-plugin/src/main/301until310-all/scala/org/apache/spark/sql/rapids/aggregate/GpuSum.scala index 4be0e643cf7..09bbf280896 100644 --- a/sql-plugin/src/main/301until310-all/scala/org/apache/spark/sql/rapids/aggregate/GpuSum.scala +++ b/sql-plugin/src/main/301until310-all/scala/org/apache/spark/sql/rapids/aggregate/GpuSum.scala @@ -16,13 +16,6 @@ package org.apache.spark.sql.rapids.aggregate -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.GpuSumBase -import org.apache.spark.sql.types.DataType - -case class GpuSum(child: Expression, - resultType: DataType, - failOnErrorOverride: Boolean = SQLConf.get.ansiEnabled) extends GpuSumBase { - override val extraDecimalOverflowChecks: Boolean = false +object GpuSumDefaults { + val hasIsEmptyField: Boolean = false } \ No newline at end of file diff --git a/sql-plugin/src/main/311+-all/scala/org/apache/spark/sql/rapids/aggregate/GpuSum.scala b/sql-plugin/src/main/311+-all/scala/org/apache/spark/sql/rapids/aggregate/GpuSum.scala index ae748013695..057f588eddf 100644 --- a/sql-plugin/src/main/311+-all/scala/org/apache/spark/sql/rapids/aggregate/GpuSum.scala +++ b/sql-plugin/src/main/311+-all/scala/org/apache/spark/sql/rapids/aggregate/GpuSum.scala @@ -16,13 +16,6 @@ package org.apache.spark.sql.rapids.aggregate -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.GpuSumBase -import org.apache.spark.sql.types.DataType - -case class GpuSum(child: Expression, - resultType: DataType, - failOnErrorOverride: Boolean = SQLConf.get.ansiEnabled) extends GpuSumBase { - override val extraDecimalOverflowChecks: Boolean = true -} \ No newline at end of file +object GpuSumDefaults { + val hasIsEmptyField: Boolean = true +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index dec844f5f10..87289552899 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -1345,24 +1345,30 @@ object GpuCast extends Arm { } } + def fixDecimalBounds(input: ColumnView, + outOfBounds: ColumnView, + ansiMode: Boolean): ColumnVector = { + if (ansiMode) { + withResource(outOfBounds.any()) { isAny => + if (isAny.isValid && isAny.getBoolean) { + throw new IllegalStateException(GpuCast.INVALID_INPUT_MESSAGE) + } + } + input.copyToColumnVector() + } else { + withResource(Scalar.fromNull(input.getType)) { nullVal => + outOfBounds.ifElse(nullVal, input) + } + } + } + def checkNFixDecimalBounds( input: ColumnView, to: DecimalType, ansiMode: Boolean): ColumnVector = { assert(input.getType.isDecimalType) withResource(DecimalUtil.outOfBounds(input, to)) { outOfBounds => - if (ansiMode) { - withResource(outOfBounds.any()) { isAny => - if (isAny.isValid && isAny.getBoolean) { - throw new IllegalStateException(GpuCast.INVALID_INPUT_MESSAGE) - } - } - input.copyToColumnVector() - } else { - withResource(Scalar.fromNull(input.getType)) { nullVal => - outOfBounds.ifElse(nullVal, input) - } - } + fixDecimalBounds(input, outOfBounds, ansiMode) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index abeb3f1fb19..d51fdb531db 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -56,7 +56,6 @@ import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.hive.rapids.GpuHiveOverrides import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids._ -import org.apache.spark.sql.rapids.aggregate.GpuSum import org.apache.spark.sql.rapids.catalyst.expressions.GpuRand import org.apache.spark.sql.rapids.execution._ import org.apache.spark.sql.rapids.execution.python._ @@ -2288,21 +2287,6 @@ object GpuOverrides extends Logging { override def tagAggForGpu(): Unit = { val inputDataType = a.child.dataType checkAndTagFloatAgg(inputDataType, conf, this) - - a.dataType match { - case _: DecimalType => - val unboundPrecision = a.child.dataType.asInstanceOf[DecimalType].precision + 10 - if (unboundPrecision > DType.DECIMAL128_MAX_PRECISION) { - if (conf.needDecimalGuarantees) { - willNotWorkOnGpu("overflow checking on sum would need " + - s"a precision of $unboundPrecision to properly detect overflows") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"sum(${a.child.dataType}) produces ${a.dataType}") - } - } - case _ => // NOOP - } } override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = @@ -3702,7 +3686,7 @@ object GpuOverrides extends Logging { exec[SampleExec]( "The backend for the sample operator", ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.DECIMAL_64).nested(), TypeSig.all), + TypeSig.ARRAY + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all), (sample, conf, p, r) => new GpuSampleExecMeta(sample, conf, p, r) ), ShimLoader.getSparkShims.aqeShuffleReaderExec, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala index aeda8d8b394..3fef0aac5ed 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala @@ -322,11 +322,13 @@ object GpuWindowExec extends Arm { // First pass replace any operations that should be totally replaced. val replacePass = expr.transformDown { case GpuWindowExpression( - GpuAggregateExpression(rep: GpuReplaceWindowFunction, _, _, _, _), spec) => + GpuAggregateExpression(rep: GpuReplaceWindowFunction, _, _, _, _), spec) + if rep.shouldReplaceWindow(spec) => // We don't actually care about the GpuAggregateExpression because it is ignored // by our GPU window operations anyways. rep.windowReplacement(spec) - case GpuWindowExpression(rep: GpuReplaceWindowFunction, spec) => + case GpuWindowExpression(rep: GpuReplaceWindowFunction, spec) + if rep.shouldReplaceWindow(spec) => rep.windowReplacement(spec) } // Second pass looks for GpuWindowFunctions and GpuWindowSpecDefinitions to build up @@ -372,6 +374,8 @@ object GpuWindowExec extends Arm { def isRunningWindow(spec: GpuWindowSpecDefinition): Boolean = spec match { case GpuWindowSpecDefinition(_, _, GpuSpecifiedWindowFrame(RowFrame, GpuSpecialFrameBoundary(UnboundedPreceding), GpuSpecialFrameBoundary(CurrentRow))) => true + case GpuWindowSpecDefinition(_, _, GpuSpecifiedWindowFrame(RowFrame, + GpuSpecialFrameBoundary(UnboundedPreceding), GpuLiteral(value, _))) if value == 0 => true case _ => false } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala index ada4000e9c4..547a6cc0d64 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala @@ -27,7 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.rapids.{GpuAggregateExpression, GpuCreateNamedStruct} +import org.apache.spark.sql.rapids.{GpuAdd, GpuAggregateExpression, GpuCreateNamedStruct} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -630,6 +630,12 @@ trait GpuReplaceWindowFunction extends GpuWindowFunction { * in the future. */ def windowReplacement(spec: GpuWindowSpecDefinition): Expression + + /** + * Return true if windowReplacement should be called to replace this GpuWindowFunction with + * something else. + */ + def shouldReplaceWindow(spec: GpuWindowSpecDefinition): Boolean = true } /** @@ -881,14 +887,17 @@ class BatchedRunningWindowBinaryFixer(val binOp: BinaryOp, val name: String) class SumBinaryFixer(toType: DataType, isAnsi: Boolean) extends BatchedRunningWindowFixer with Arm with Logging { private val name = "sum" - private val binOp = BinaryOp.ADD private var previousResult: Option[Scalar] = None + private var previousOverflow: Option[Scalar] = None - def updateState(finalOutputColumn: cudf.ColumnVector): Unit = { + def updateState(finalOutputColumn: cudf.ColumnVector, + wasOverflow: Option[cudf.ColumnVector]): Unit = { + val lastIndex = finalOutputColumn.getRowCount.toInt - 1 logDebug(s"$name: updateState from $previousResult to...") previousResult.foreach(_.close) - previousResult = - Some(finalOutputColumn.getScalarElement(finalOutputColumn.getRowCount.toInt - 1)) + previousResult = Some(finalOutputColumn.getScalarElement(lastIndex)) + previousOverflow.foreach(_.close()) + previousOverflow = wasOverflow.map(_.getScalarElement(lastIndex)) logDebug(s"$name: ... $previousResult") } @@ -911,8 +920,7 @@ class SumBinaryFixer(toType: DataType, isAnsi: Boolean) throw new IllegalArgumentException(s"Making a zero scalar for $other is not supported") } - override def fixUp(samePartitionMask: Either[cudf.ColumnVector, Boolean], - sameOrderMask: Option[Either[cudf.ColumnVector, Boolean]], + private[this] def fixUpNonDecimal(samePartitionMask: Either[cudf.ColumnVector, Boolean], windowedColumnOutput: cudf.ColumnView): cudf.ColumnVector = { logDebug(s"$name: fix up $previousResult $samePartitionMask") val ret = (previousResult, samePartitionMask) match { @@ -928,7 +936,7 @@ class SumBinaryFixer(toType: DataType, isAnsi: Boolean) } } withResource(nullsReplaced) { nullsReplaced => - nullsReplaced.binaryOp(binOp, prev, prev.getType) + nullsReplaced.binaryOp(BinaryOp.ADD, prev, prev.getType) } } else { // prev is NULL but NULL + something == NULL which we don't want @@ -948,7 +956,7 @@ class SumBinaryFixer(toType: DataType, isAnsi: Boolean) } } withResource(nullsReplaced) { nullsReplaced => - withResource(nullsReplaced.binaryOp(binOp, prev, prev.getType)) { updated => + withResource(nullsReplaced.binaryOp(BinaryOp.ADD, prev, prev.getType)) { updated => mask.ifElse(updated, windowedColumnOutput) } } @@ -958,15 +966,126 @@ class SumBinaryFixer(toType: DataType, isAnsi: Boolean) } } closeOnExcept(ret) { ret => - updateState(ret) + updateState(ret, None) + ret + } + } + + private[this] def fixUpDecimal(samePartitionMask: Either[cudf.ColumnVector, Boolean], + windowedColumnOutput: cudf.ColumnView, + dt: DecimalType): cudf.ColumnVector = { + logDebug(s"$name: fix up $previousResult $samePartitionMask") + val (ret, decimalOverflowOnAdd) = (previousResult, previousOverflow, samePartitionMask) match { + case (None, None, _) => + // The mask is all false so do nothing + withResource(Scalar.fromBool(false)) { falseVal => + closeOnExcept(ColumnVector.fromScalar(falseVal, + windowedColumnOutput.getRowCount.toInt)) { over => + (incRef(windowedColumnOutput), over) + } + } + case (Some(prev), Some(previousOver), scala.util.Right(mask)) => + if (mask) { + if (!prev.isValid) { + // So in the window operation we can have a null if all of the input values before it + // were also null or if we overflowed the result and inserted in a null. + // + // If we overflowed, then all of the output for this group should be null, but the + // overflow check code can handle inserting that, so just inc the ref count and return + // the overflow column. + // + // If we didn't overflow, and the input is null then + // prev is NULL but NULL + something == NULL which we don't want, so also + // just increment the reference count and go on. + closeOnExcept(ColumnVector.fromScalar(previousOver, + windowedColumnOutput.getRowCount.toInt)) { over => + (incRef(windowedColumnOutput), over) + } + } else { + // The previous didn't overflow, so now we need to do the add and check for overflow. + val nullsReplaced = withResource(windowedColumnOutput.isNull) { nulls => + withResource(makeZeroScalar(windowedColumnOutput.getType)) { zero => + nulls.ifElse(zero, windowedColumnOutput) + } + } + withResource(nullsReplaced) { nullsReplaced => + closeOnExcept(nullsReplaced.binaryOp(BinaryOp.ADD, prev, prev.getType)) { added => + (added, GpuAdd.didDecimalOverflow(nullsReplaced, prev, added)) + } + } + } + } else { + // The mask is all false so do nothing + withResource(Scalar.fromBool(false)) { falseVal => + closeOnExcept(ColumnVector.fromScalar(falseVal, + windowedColumnOutput.getRowCount.toInt)) { over => + (incRef(windowedColumnOutput), over) + } + } + } + case (Some(prev), Some(previousOver), scala.util.Left(mask)) => + if (prev.isValid) { + // The previous didn't overflow, so now we need to do the add and check for overflow. + val nullsReplaced = withResource(windowedColumnOutput.isNull) { nulls => + withResource(nulls.and(mask)) { shouldReplace => + withResource(makeZeroScalar(windowedColumnOutput.getType)) { zero => + shouldReplace.ifElse(zero, windowedColumnOutput) + } + } + } + withResource(nullsReplaced) { nullsReplaced => + withResource(nullsReplaced.binaryOp(BinaryOp.ADD, prev, prev.getType)) { added => + closeOnExcept(mask.ifElse(added, windowedColumnOutput)) { updated => + withResource(Scalar.fromBool(false)) { falseVal => + withResource(GpuAdd.didDecimalOverflow(nullsReplaced, prev, added)) { over => + (updated, mask.ifElse(over, falseVal)) + } + } + } + } + } + } else { + // So in the window operation we can have a null if all of the input values before it + // were also null or if we overflowed the result and inserted in a null. + // + // If we overflowed, then all of the output for this group should be null, but the + // overflow check code can handle inserting that, so just inc the ref count and return + // the overflow column. + // + // If we didn't overflow, and the input is null then + // prev is NULL but NULL + something == NULL which we don't want, so also + // just increment the reference count and go on. + closeOnExcept(ColumnVector.fromScalar(previousOver, + windowedColumnOutput.getRowCount.toInt)) { over => + (incRef(windowedColumnOutput), over) + } + } + case _ => + throw new IllegalStateException("INTERNAL ERROR: Should never have a situation where " + + "prev and previousOver do not match.") } + withResource(ret) { ret => + withResource(decimalOverflowOnAdd) { decimalOverflowOnAdd => + withResource(DecimalUtil.outOfBounds(ret, dt)) { valOutOfBounds => + withResource(valOutOfBounds.or(decimalOverflowOnAdd)) { outOfBounds => + closeOnExcept(GpuCast.fixDecimalBounds(ret, outOfBounds, isAnsi)) { replaced => + updateState(replaced, Some(outOfBounds)) + replaced + } + } + } + } + } + } + + override def fixUp(samePartitionMask: Either[cudf.ColumnVector, Boolean], + sameOrderMask: Option[Either[cudf.ColumnVector, Boolean]], + windowedColumnOutput: cudf.ColumnView): cudf.ColumnVector = { toType match { case dt: DecimalType => - withResource(ret) { ret => - GpuCast.checkNFixDecimalBounds(ret, dt, isAnsi) - } + fixUpDecimal(samePartitionMask, windowedColumnOutput, dt) case _ => - ret + fixUpNonDecimal(samePartitionMask, windowedColumnOutput) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index 448ddfe6be6..58896717179 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -188,6 +188,7 @@ class GpuHashAggregateIterator( metrics: GpuHashAggregateMetrics, configuredTargetBatchSize: Long) extends Iterator[ColumnarBatch] with Arm with AutoCloseable with Logging { + // Partial mode: // 1. boundInputReferences: picks column from raw input // 2. boundFinalProjections: is a pass-through of the agg buffer @@ -205,7 +206,6 @@ class GpuHashAggregateIterator( // (GpuAverage => CudfSum/CudfCount) // 3. boundResultReferences: project the result expressions Spark expects in the output. private case class BoundExpressionsModeAggregates( - boundInputReferences: Seq[GpuExpression], boundFinalProjections: Option[Seq[GpuExpression]], boundResultReferences: Seq[Expression]) @@ -277,13 +277,10 @@ class GpuHashAggregateIterator( /** Aggregate all input batches and place the results in the aggregatedBatches queue. */ private def aggregateInputBatches(): Unit = { - val aggHelper = new AggHelper(merge = false) + val aggHelper = new AggHelper(forceMerge = false) while (cbIter.hasNext) { - val (childBatch, isLastInputBatch) = withResource(cbIter.next()) { inputBatch => - val isLast = GpuColumnVector.isTaggedAsFinalBatch(inputBatch) - (processIncomingBatch(inputBatch), isLast) - } - withResource(childBatch) { _ => + withResource(cbIter.next()) { childBatch => + val isLastInputBatch = GpuColumnVector.isTaggedAsFinalBatch(childBatch) withResource(computeAggregate(childBatch, aggHelper)) { aggBatch => val batch = LazySpillableColumnarBatch(aggBatch, metrics.spillCallback, "aggbatch") // Avoid making batch spillable for the common case of the last and only batch @@ -385,7 +382,7 @@ class GpuHashAggregateIterator( wasBatchMerged } - private lazy val concatAndMergeHelper = new AggHelper(merge = true) + private lazy val concatAndMergeHelper = new AggHelper(forceMerge = true) /** * Concatenate batches together and perform a merge aggregation on the result. The input batches @@ -544,27 +541,6 @@ class GpuHashAggregateIterator( } } - /** Perform the initial projection on the input batch and extract the result columns */ - private def processIncomingBatch(batch: ColumnarBatch): ColumnarBatch = { - val aggTime = metrics.computeAggTime - val opTime = metrics.opTime - withResource(new NvtxWithMetrics("prep agg batch", NvtxColor.CYAN, aggTime, - opTime)) { _ => - val cols = boundExpressions.boundInputReferences.safeMap { ref => - val childCv = GpuExpressionsUtils.columnarEvalToColumn(ref, batch) - if (DataType.equalsStructurally(childCv.dataType, ref.dataType, ignoreNullability = true)) { - childCv - } else { - withResource(childCv) { childCv => - val rapidsType = GpuColumnVector.getNonNestedRapidsType(ref.dataType) - GpuColumnVector.from(childCv.getBase.castTo(rapidsType), ref.dataType) - } - } - } - new ColumnarBatch(cols.toArray, batch.numRows()) - } - } - /** * Concatenates batches after extracting them from `LazySpillableColumnarBatch` * @note the input batches are not closed as part of this operation @@ -603,30 +579,6 @@ class GpuHashAggregateIterator( val aggBufferAttributes = groupingAttributes ++ aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) - // Adapted from `AggregationIterator.initializeAggregateFunctions` in Spark: - // - we use the "imperative aggregate" way as it used bound expressions due to - // lack of support of codegen (like our case) - // - for partial/complete: we bind to the inputProjection as specified by each - // `GpuAggregateFunction` to the `inputAttributes` (see how those are defined) - // - for partial merge/final: it is the pass through case, we are getting as input - // the "agg buffer", and we are using `inputAggBufferAttributes` to match the Spark - // function. We still bind to `inputAttributes`, as those would be setup for pass-through - // in the partial merge/final cases. - val aggBound = aggregateExpressions.flatMap { agg => - agg.mode match { - case Partial | Complete => - agg.aggregateFunction.inputProjection - case PartialMerge | Final => - agg.aggregateFunction.inputAggBufferAttributes - case mode => - throw new NotImplementedError(s"can't translate ${mode}") - } - } - - val boundInputReferences = GpuBindReferences.bindGpuReferences( - groupingExpressions ++ aggBound, - inputAttributes) - val boundFinalProjections = if (modeInfo.hasFinalMode || modeInfo.hasCompleteMode) { val finalProjections = groupingExpressions ++ aggregateExpressions.map(_.aggregateFunction.evaluateExpression) @@ -659,7 +611,6 @@ class GpuHashAggregateIterator( groupingAttributes) } BoundExpressionsModeAggregates( - boundInputReferences, boundFinalProjections, boundResultReferences) } @@ -667,12 +618,12 @@ class GpuHashAggregateIterator( /** * Internal class used in `computeAggregates` for the pre, agg, and post steps * - * @param merge - if true, we are merging two pre-aggregated batches, so we should use + * @param forceMerge - if true, we are merging two pre-aggregated batches, so we should use * the merge steps for each aggregate function * @param isSorted - if the batch is sorted this is set to true and is passed to cuDF * as an optimization hint */ - class AggHelper(merge: Boolean, isSorted: Boolean = false) { + class AggHelper(forceMerge: Boolean, isSorted: Boolean = false) { // `CudfAggregate` instances to apply, either update or merge aggregates private val cudfAggregates = new mutable.ArrayBuffer[CudfAggregate]() @@ -694,10 +645,8 @@ class GpuHashAggregateIterator( private val postStepAttr = new mutable.ArrayBuffer[Attribute]() // we add the grouping expression first, which bind as pass-through - preStep ++= GpuBindReferences.bindGpuReferences( - groupingAttributes, groupingAttributes) - postStep ++= GpuBindReferences.bindGpuReferences( - groupingAttributes, groupingAttributes) + preStep ++= groupingExpressions + postStep ++= groupingAttributes postStepAttr ++= groupingAttributes postStepDataTypes ++= groupingExpressions.map(_.dataType) @@ -705,14 +654,14 @@ class GpuHashAggregateIterator( private var ix = groupingAttributes.length for (aggExp <- aggregateExpressions) { val aggFn = aggExp.aggregateFunction - if ((aggExp.mode == Partial || aggExp.mode == Complete) && !merge) { + if ((aggExp.mode == Partial || aggExp.mode == Complete) && !forceMerge) { val ordinals = (ix until ix + aggFn.updateAggregates.length) aggOrdinals ++= ordinals ix += ordinals.length val updateAggs = aggFn.updateAggregates postStepDataTypes ++= updateAggs.map(_.dataType) cudfAggregates ++= updateAggs - preStep ++= aggFn.aggBufferAttributes + preStep ++= aggFn.inputProjection postStep ++= aggFn.postUpdate postStepAttr ++= aggFn.postUpdateAttr } else { @@ -729,8 +678,11 @@ class GpuHashAggregateIterator( } // a bound expression that is applied before the cuDF aggregate - private val preStepBound = + private val preStepBound = if (forceMerge) { GpuBindReferences.bindGpuReferences(preStep, aggBufferAttributes) + } else { + GpuBindReferences.bindGpuReferences(preStep, inputAttributes) + } // a bound expression that is applied after the cuDF aggregate private val postStepBound = @@ -1388,11 +1340,11 @@ case class GpuHashAggregateExec( private val inputAggBufferAttributes: Seq[Attribute] = { aggregateExpressions - // there're exactly four cases needs `inputAggBufferAttributes` from child according to the - // agg planning in `AggUtils`: Partial -> Final, PartialMerge -> Final, - // Partial -> PartialMerge, PartialMerge -> PartialMerge. - .filter(a => a.mode == Final || a.mode == PartialMerge) - .flatMap(_.aggregateFunction.inputAggBufferAttributes) + // there're exactly four cases needs `inputAggBufferAttributes` from child according to the + // agg planning in `AggUtils`: Partial -> Final, PartialMerge -> Final, + // Partial -> PartialMerge, PartialMerge -> PartialMerge. + .filter(a => a.mode == Final || a.mode == PartialMerge) + .flatMap(_.aggregateFunction.aggBufferAttributes) } private lazy val uniqueModes: Seq[AggregateMode] = aggregateExpressions.map(_.mode).distinct @@ -1519,7 +1471,7 @@ case class GpuHashAggregateExec( */ override lazy val allAttributes: AttributeSeq = child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++ - aggregateExpressions.flatMap(_.aggregateFunction.inputAggBufferAttributes) + aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index c9fc473e221..55953cc1b7c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -28,7 +28,8 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeS import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.util.{ArrayData, TypeUtils} -import org.apache.spark.sql.rapids.aggregate.GpuSum +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.aggregate.GpuSumDefaults import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -176,14 +177,12 @@ trait GpuAggregateFunction extends GpuExpression */ val evaluateExpression: Expression - /** Attributes of fields in aggBufferSchema. */ - def aggBufferAttributes: Seq[AttributeReference] - /** - * Result of the aggregate function when the input is empty. This is currently only used for the - * proper rewriting of distinct aggregate functions. + * This is the contract with the outside world. It describes what the output of postUpdate should + * look like, and what the input to preMerge looks like. It also describes what the output of + * postMerge must look like. */ - def defaultResult: Option[GpuLiteral] = None + def aggBufferAttributes: Seq[AttributeReference] def sql(isDistinct: Boolean): String = { val distinct = if (isDistinct) "DISTINCT " else "" @@ -196,19 +195,8 @@ trait GpuAggregateFunction extends GpuExpression prettyName + flatArguments.mkString(start, ", ", ")") } - /** - * Attributes of fields in input aggregation buffers (immutable aggregation buffers that are - * merged with mutable aggregation buffers in the merge() function or merge expressions). - * These attributes are created automatically by cloning the [[aggBufferAttributes]]. - */ - final lazy val inputAggBufferAttributes: Seq[AttributeReference] = - aggBufferAttributes.map(_.newInstance()) - /** An aggregate function is not foldable. */ final override def foldable: Boolean = false - - /** The schema of the aggregation buffer. */ - def aggBufferSchema: StructType = null //not used in GPU version } case class WrappedAggFunction(aggregateFunction: GpuAggregateFunction, filter: Expression) @@ -575,6 +563,75 @@ case class GpuMax(child: Expression) extends GpuAggregateFunction } } +/** + * All decimal processing in Spark has overflow detection as a part of it. Either it replaces + * the value with a null in non-ANSI mode, or it throws an exception in ANSI mode. Spark will also + * do the processing for larger values as `Decimal` values which are based on `BigDecimal` and have + * unbounded precision. So in most cases it is impossible to overflow/underflow so much that an + * incorrect value is returned. Spark will just use more and more memory to hold the value and + * then check for overflow at some point when the result needs to be turned back into a 128-bit + * value. + * + * We cannot do the same thing. Instead we take three strategies to detect overflow. + * + * 1. For decimal values with a precision of 8 or under we follow Spark and do the SUM + * on the unscaled value as a long, and then bit-cast the result back to a Decimal value. + * this means that we can SUM `174,467,442,481` maximum or minimum decimal values with a + * precision of 8 before overflow can no longer be detected. It is much higher for decimal + * values with a smaller precision. + * 2. For decimal values with a precision from 9 to 20 inclusive we sum them as 128-bit values. + * this is very similar to what we do in the first strategy. The main differences are that we + * use a 128-bit value when doing the sum, and we check for overflow after processing each batch. + * In the case of group-by and reduction that happens after the update stage and also after each + * merge stage. This gives us enough room that we can always detect overflow when summing a + * single batch. Even on a merge where we could be doing the aggregation on a batch that has + * all max output values in it. + * 3. For values from 21 to 28 inclusive we have enough room to not check for overflow on teh update + * aggregation, but for the merge aggregation we need to do some extra checks. This is done by + * taking the digits above 28 and sum them separately. We then check to see if they would have + * overflowed the original limits. This lets us detect overflow in cases where the original + * value would have wrapped around. The reason this works is because we have a hard limit on the + * maximum number of values in a single batch being processed. `Int.MaxValue`, or about 2.2 + * billion values. So we use a precision on the higher values that is large enough to handle + * 2.2 billion values and still detect overflow. This equates to a precision of about 10 more + * than is needed to hold the higher digits. This effectively gives us unlimited overflow + * detection. + * 4. For anything larger than precision 28 we do the same overflow detection for strategy 3, but + * also do it on the update aggregation. This lets us fully detect overflows in any stage of + * an aggregation. + * + * Note that for Window operations either there is no merge stage or it only has a single value + * being merged into a batch instead of an entire batch being merged together. This lets us handle + * the overflow detection with what is built into GpuAdd. + */ +object GpuDecimalSumOverflow { + /** + * The increase in precision for the output of a SUM from the input. This is hard coded by + * Spark so we just have it here. This means that for most types without being limited to + * a precision of 38 you get 10-billion+ values before an overflow would even be possible. + */ + val sumPrecisionIncrease: Int = 10 + + /** + * Generally we want a guarantee that is at least 10x larger than the original overflow. + */ + val extraGuaranteePrecision: Int = 1 + + /** + * The precision above which we need extra overflow checks while doing an update. This is because + * anything above this precision could in theory overflow beyond detection within a single input + * batch. + */ + val updateCutoffPrecision: Int = 28 + + /** + * This is the precision above which we need to do extra checks for overflow when merging + * results. This is because anything above this precision could in theory overflow a decimal128 + * value beyond detection in a batch of already updated and checked values. + */ + val mergeCutoffPrecision: Int = 20 +} + /** * This is equivalent to what Spark does after a sum to check for overflow * ` @@ -627,15 +684,122 @@ case class GpuCheckOverflowAfterSum( override def children: Seq[Expression] = Seq(data, isEmpty) } -trait GpuSumBase extends GpuAggregateFunction with ImplicitCastInputTypes - with GpuBatchedRunningWindowWithFixer - with GpuAggregateWindowFunction - with GpuRunningWindowFunction { +/** + * This extracts the highest digits from a Decimal value as a part of doing a SUM. + */ +case class GpuDecimalSumHighDigits( + input: Expression, + originalInputType: DecimalType) extends GpuExpression with ShimExpression { + + override def nullable: Boolean = input.nullable + + override def toString: String = s"GpuDecimalSumHighDigits($input)" + + override def sql: String = input.sql + + override val dataType: DecimalType = DecimalType(originalInputType.precision + + GpuDecimalSumOverflow.sumPrecisionIncrease + GpuDecimalSumOverflow.extraGuaranteePrecision - + GpuDecimalSumOverflow.updateCutoffPrecision, 0) + // Marking these as lazy because they are not serializable + private lazy val outputDType = GpuColumnVector.getNonNestedRapidsType(dataType) + private lazy val intermediateDType = + DType.create(DType.DTypeEnum.DECIMAL128, outputDType.getScale) + + private lazy val divisionFactor: Decimal = + Decimal(math.pow(10, GpuDecimalSumOverflow.updateCutoffPrecision)) + private val divisionType = DecimalType(38, 0) + + override def columnarEval(batch: ColumnarBatch): Any = { + withResource(GpuProjectExec.projectSingle(batch, input)) { inputCol => + val inputBase = inputCol.getBase + // We don't have direct access to 128 bit ints so we use a decimal with a scale of 0 + // as a stand in. + val bitCastInputType = DType.create(DType.DTypeEnum.DECIMAL128, 0) + val divided = withResource(inputBase.bitCastTo(bitCastInputType)) { bitCastInput => + withResource(GpuScalar.from(divisionFactor, divisionType)) { divisor => + bitCastInput.div(divisor, intermediateDType) + } + } + val ret = withResource(divided) { divided => + if (divided.getType.equals(outputDType)) { + divided.incRefCount() + } else { + divided.castTo(outputDType) + } + } + GpuColumnVector.from(ret, dataType) + } + } + + override def children: Seq[Expression] = Seq(input) +} + +/** + * Return a boolean if this decimal overflowed or not + */ +case class GpuDecimalDidOverflow( + data: Expression, + rangeType: DecimalType, + nullOnOverflow: Boolean) extends GpuExpression with ShimExpression { - val child: Expression - val resultType: DataType - val failOnErrorOverride: Boolean - val extraDecimalOverflowChecks: Boolean + override def nullable: Boolean = true + + override def toString: String = + s"GpuDecimalDidOverflow($data, $rangeType, $nullOnOverflow)" + + override def sql: String = data.sql + + override def dataType: DataType = BooleanType + + override def columnarEval(batch: ColumnarBatch): Any = { + withResource(GpuProjectExec.projectSingle(batch, data)) { dataCol => + val dataBase = dataCol.getBase + withResource(DecimalUtil.outOfBounds(dataBase, rangeType)) { outOfBounds => + if (!nullOnOverflow) { + withResource(outOfBounds.any()) { isAny => + if (isAny.isValid && isAny.getBoolean) { + throw new ArithmeticException("Overflow as a part of SUM") + } + } + } else { + GpuColumnVector.from(outOfBounds.incRefCount(), dataType) + } + } + } + } + + override def children: Seq[Expression] = Seq(data) +} + +case class GpuSum(child: Expression, + resultType: DataType, + failOnErrorOverride: Boolean = SQLConf.get.ansiEnabled, + forceWindowSumToNotBeReplaced: Boolean = false) + extends GpuAggregateFunction with ImplicitCastInputTypes + with GpuReplaceWindowFunction + with GpuBatchedRunningWindowWithFixer + with GpuAggregateWindowFunction + with GpuRunningWindowFunction { + + private lazy val childIsDecimal: Boolean = + child.dataType.isInstanceOf[DecimalType] + + private lazy val childDecimalType: DecimalType = + child.dataType.asInstanceOf[DecimalType] + + private lazy val needsDec128MergeOverflowChecks: Boolean = + childIsDecimal && childDecimalType.precision > GpuDecimalSumOverflow.mergeCutoffPrecision + + private lazy val needsDec128UpdateOverflowChecks: Boolean = + childIsDecimal && + childDecimalType.precision > GpuDecimalSumOverflow.updateCutoffPrecision + + // For some operations we need to SUm the higher digits in addition to the regular value so + // we can detect overflow. This is the type of the higher digits SUM value. + private lazy val higherDigitsCheckType: DecimalType = { + val t = resultType.asInstanceOf[DecimalType] + DecimalType(t.precision - GpuDecimalSumOverflow.updateCutoffPrecision, 0) + } private lazy val zeroDec = { val dt = resultType.asInstanceOf[DecimalType] @@ -643,74 +807,198 @@ trait GpuSumBase extends GpuAggregateFunction with ImplicitCastInputTypes } override lazy val initialValues: Seq[GpuLiteral] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => + case _: DecimalType if GpuSumDefaults.hasIsEmptyField => Seq(zeroDec, GpuLiteral(true, BooleanType)) case _ => Seq(GpuLiteral(null, resultType)) } + private lazy val updateHigherOrderBits = { + val input = if (child.dataType != resultType) { + GpuCast(child, resultType) + } else { + child + } + GpuDecimalSumHighDigits(input, childDecimalType) + } + // we need to cast to `resultType` here, since Spark is not widening types // as done before Spark 3.2.0. See CudfSum for more info. override lazy val inputProjection: Seq[Expression] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - // Spark tracks null columns through a second column isEmpty for decimal. - Seq(GpuIf(GpuIsNull(child), zeroDec, GpuCast(child, resultType)), GpuIsNull(child)) + case _: DecimalType => + // Decimal is complicated... + if (GpuSumDefaults.hasIsEmptyField) { + // Spark tracks null columns through a second column isEmpty for decimal. So null values + // are replaced with 0, and a separate boolean column for isNull is added + if (needsDec128UpdateOverflowChecks) { + // If we want extra checks for overflow, then we also want to include it here + Seq(GpuIf(GpuIsNull(child), zeroDec, GpuCast(child, resultType)), + GpuIsNull(child), + updateHigherOrderBits) + } else { + Seq(GpuIf(GpuIsNull(child), zeroDec, GpuCast(child, resultType)), GpuIsNull(child)) + } + } else { + if (needsDec128UpdateOverflowChecks) { + // If we want extra checks for overflow, then we also want to include it here + Seq(GpuCast(child, resultType), updateHigherOrderBits) + } else { + Seq(GpuCast(child, resultType)) + } + } case _ => Seq(GpuCast(child, resultType)) } private lazy val updateSum = new CudfSum(resultType) private lazy val updateIsEmpty = new CudfMin(BooleanType) + private lazy val updateOverflow = new CudfSum(updateHigherOrderBits.dataType) override lazy val updateAggregates: Seq[CudfAggregate] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - Seq(updateSum, updateIsEmpty) + case _: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128UpdateOverflowChecks) { + Seq(updateSum, updateIsEmpty, updateOverflow) + } else { + Seq(updateSum, updateIsEmpty) + } + } else { + if (needsDec128UpdateOverflowChecks) { + Seq(updateSum, updateOverflow) + } else { + Seq(updateSum) + } + } case _ => Seq(updateSum) } + private[this] def extendedPostUpdateDecOverflowCheck(dt: DecimalType) = + GpuCheckOverflow( + GpuIf( + GpuDecimalDidOverflow(updateOverflow.attr, + higherDigitsCheckType, + !failOnErrorOverride), + GpuLiteral(null, dt), + updateSum.attr), + dt, !failOnErrorOverride) + override lazy val postUpdate: Seq[Expression] = resultType match { - case dt: DecimalType if extraDecimalOverflowChecks => - Seq(GpuCheckOverflow(updateSum.attr, dt, !failOnErrorOverride), updateIsEmpty.attr) + case dt: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128UpdateOverflowChecks) { + Seq(extendedPostUpdateDecOverflowCheck(dt), updateIsEmpty.attr) + } else { + Seq(GpuCheckOverflow(updateSum.attr, dt, !failOnErrorOverride), updateIsEmpty.attr) + } + } else { + if (needsDec128UpdateOverflowChecks) { + Seq(extendedPostUpdateDecOverflowCheck(dt)) + } else { + postUpdateAttr + } + } case _ => postUpdateAttr } // output of GpuSum private lazy val sum = AttributeReference("sum", resultType)() + // Used for Decimal overflow detection private lazy val isEmpty = AttributeReference("isEmpty", BooleanType)() override lazy val aggBufferAttributes: Seq[AttributeReference] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => + case _: DecimalType if GpuSumDefaults.hasIsEmptyField => sum :: isEmpty :: Nil case _ => sum :: Nil } + private lazy val mergeHigherOrderBits = GpuDecimalSumHighDigits(sum, childDecimalType) + override lazy val preMerge: Seq[Expression] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - Seq(sum, isEmpty, GpuIsNull(sum)) + case _: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128MergeOverflowChecks) { + Seq(sum, isEmpty, GpuIsNull(sum), mergeHigherOrderBits) + } else { + Seq(sum, isEmpty, GpuIsNull(sum)) + } + } else { + if (needsDec128MergeOverflowChecks) { + Seq(sum, mergeHigherOrderBits) + } else { + aggBufferAttributes + } + } case _ => aggBufferAttributes } private lazy val mergeSum = new CudfSum(resultType) private lazy val mergeIsEmpty = new CudfMin(BooleanType) private lazy val mergeIsOverflow = new CudfMax(BooleanType) + private lazy val mergeOverflow = new CudfSum(mergeHigherOrderBits.dataType) // To be able to do decimal overflow detection, we need a CudfSum that does **not** ignore nulls. // Cudf does not have such an aggregation, so for merge we have to work around that similar to // what happens with isEmpty override lazy val mergeAggregates: Seq[CudfAggregate] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - Seq(mergeSum, mergeIsEmpty, mergeIsOverflow) + case _: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128MergeOverflowChecks) { + Seq(mergeSum, mergeIsEmpty, mergeIsOverflow, mergeOverflow) + } else { + Seq(mergeSum, mergeIsEmpty, mergeIsOverflow) + } + } else { + if (needsDec128MergeOverflowChecks) { + Seq(mergeSum, mergeOverflow) + } else { + Seq(mergeSum) + } + } case _ => Seq(mergeSum) } override lazy val postMerge: Seq[Expression] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - Seq(GpuIf(mergeIsOverflow.attr, GpuLiteral.create(null, resultType), mergeSum.attr), - mergeIsEmpty.attr) + case dt: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128MergeOverflowChecks) { + Seq( + GpuCheckOverflow( + GpuIf( + GpuOr( + GpuDecimalDidOverflow(mergeOverflow.attr, higherDigitsCheckType, + !failOnErrorOverride), + mergeIsOverflow.attr), + GpuLiteral.create(null, resultType), + mergeSum.attr), + dt, !failOnErrorOverride), + mergeIsEmpty.attr) + } else { + Seq( + GpuCheckOverflow(GpuIf(mergeIsOverflow.attr, + GpuLiteral.create(null, resultType), + mergeSum.attr), + dt, !failOnErrorOverride), + mergeIsEmpty.attr) + } + } else { + if (needsDec128MergeOverflowChecks) { + Seq( + GpuCheckOverflow( + GpuIf( + GpuDecimalDidOverflow(mergeOverflow.attr, higherDigitsCheckType, + !failOnErrorOverride), + GpuLiteral.create(null, resultType), + mergeSum.attr), + dt, !failOnErrorOverride)) + } else { + postMergeAttr + } + } + case _ => postMergeAttr } override lazy val evaluateExpression: Expression = resultType match { case dt: DecimalType => - if (extraDecimalOverflowChecks) { + if (GpuSumDefaults.hasIsEmptyField) { GpuCheckOverflowAfterSum(sum, isEmpty, dt, !failOnErrorOverride) } else { GpuCheckOverflow(sum, dt, !failOnErrorOverride) @@ -726,6 +1014,32 @@ trait GpuSumBase extends GpuAggregateFunction with ImplicitCastInputTypes override def checkInputDataTypes(): TypeCheckResult = TypeUtils.checkForNumericExpr(child.dataType, "function gpu sum") + // Replacement Window Function + override def shouldReplaceWindow(spec: GpuWindowSpecDefinition): Boolean = { + // We only will replace this if we think an update will fail. In the cases where we can + // handle a window function larger than a single batch, we already have merge overflow + // detection enabled. + !forceWindowSumToNotBeReplaced && needsDec128UpdateOverflowChecks + } + + override def windowReplacement(spec: GpuWindowSpecDefinition): Expression = { + // We need extra overflow checks for some larger decimal type. To do these checks we + // extract the higher digits and SUM them separately to see if they would overflow. + // If they do we know that the regular SUM also overflowed. If not we know that we can rely on + // the existing overflow code to detect it. + val regularSum = GpuWindowExpression( + GpuSum(child, resultType, failOnErrorOverride = failOnErrorOverride, + forceWindowSumToNotBeReplaced = true), + spec) + val highOrderDigitsSum = GpuWindowExpression( + GpuSum( + GpuDecimalSumHighDigits(GpuCast(child, resultType), childDecimalType), + higherDigitsCheckType, + failOnErrorOverride = failOnErrorOverride), + spec) + GpuIf(GpuIsNull(highOrderDigitsSum), GpuLiteral(null, resultType), regularSum) + } + // GENERAL WINDOW FUNCTION // Spark 3.2.0+ stopped casting the input data to the output type before the sum operation // This fixes that. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala index a349c875965..86de80c5703 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -144,17 +144,8 @@ abstract class CudfBinaryArithmetic extends CudfBinaryOperator with NullIntolera override def dataType: DataType = left.dataType } -case class GpuAdd( - left: Expression, - right: Expression, - failOnError: Boolean) extends CudfBinaryArithmetic { - override def inputType: AbstractDataType = TypeCollection.NumericAndInterval - - override def symbol: String = "+" - - override def binaryOp: BinaryOp = BinaryOp.ADD - - private[this] def basicOpOverflowCheck( +object GpuAdd extends Arm { + def basicOpOverflowCheck( lhs: BinaryOperable, rhs: BinaryOperable, ret: ColumnVector): Unit = { @@ -179,7 +170,7 @@ case class GpuAdd( } } - private[this] def decimalOpOverflowCheck( + def didDecimalOverflow( lhs: BinaryOperable, rhs: BinaryOperable, ret: ColumnVector): ColumnVector = { @@ -189,7 +180,7 @@ case class GpuAdd( // the result val numRows = ret.getRowCount.toInt val zero = BigDecimal(0) - val overflow = withResource(DecimalUtil.lessThan(rhs, zero, numRows)) { rhsLz => + withResource(DecimalUtil.lessThan(rhs, zero, numRows)) { rhsLz => val argsSignSame = withResource(DecimalUtil.lessThan(lhs, zero, numRows)) { lhsLz => lhsLz.equalTo(rhsLz) } @@ -203,7 +194,14 @@ case class GpuAdd( } } } - withResource(overflow) { overflow => + } + + def decimalOpOverflowCheck( + lhs: BinaryOperable, + rhs: BinaryOperable, + ret: ColumnVector, + failOnError: Boolean): ColumnVector = { + withResource(didDecimalOverflow(lhs, rhs, ret)) { overflow => if (failOnError) { withResource(overflow.any()) { any => if (any.isValid && any.getBoolean) { @@ -212,23 +210,34 @@ case class GpuAdd( } ret.incRefCount() } else { - withResource(GpuScalar.from(null, dataType)) { nullVal => + withResource(Scalar.fromNull(ret.getType)) { nullVal => overflow.ifElse(nullVal, ret) } } } } +} + +case class GpuAdd( + left: Expression, + right: Expression, + failOnError: Boolean) extends CudfBinaryArithmetic { + override def inputType: AbstractDataType = TypeCollection.NumericAndInterval + + override def symbol: String = "+" + + override def binaryOp: BinaryOp = BinaryOp.ADD override def doColumnar(lhs: BinaryOperable, rhs: BinaryOperable): ColumnVector = { val ret = super.doColumnar(lhs, rhs) withResource(ret) { ret => // No shims are needed, because it actually supports ANSI mode from Spark v3.0.1. if (failOnError && GpuAnsi.needBasicOpOverflowCheck(dataType)) { - basicOpOverflowCheck(lhs, rhs, ret) + GpuAdd.basicOpOverflowCheck(lhs, rhs, ret) } if (dataType.isInstanceOf[DecimalType]) { - decimalOpOverflowCheck(lhs, rhs, ret) + GpuAdd.decimalOpOverflowCheck(lhs, rhs, ret, failOnError) } else { ret.incRefCount() } From 262c785192e5d443a44fb88a9375a02820e6ccd3 Mon Sep 17 00:00:00 2001 From: Hao Zhu <9665750+viadea@users.noreply.github.com> Date: Mon, 6 Dec 2021 16:21:49 -0800 Subject: [PATCH 28/52] Update Download page for 21.12 [skip ci] (#4307) * Update Download page for 21.12 Signed-off-by: Hao Zhu * Update docs/download.md Co-authored-by: Jason Lowe * Update docs/download.md Co-authored-by: Jason Lowe * Update docs/download.md Co-authored-by: Jason Lowe * Update docs/download.md Co-authored-by: Jason Lowe * Change cuda version to 11.x Signed-off-by: Hao Zhu * Update docs/download.md Co-authored-by: Jason Lowe * Update docs/download.md Co-authored-by: Sameer Raheja Co-authored-by: Jason Lowe Co-authored-by: Sameer Raheja --- docs/download.md | 66 ++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 66 insertions(+) diff --git a/docs/download.md b/docs/download.md index 70363c50ad7..b92b44ba64b 100644 --- a/docs/download.md +++ b/docs/download.md @@ -18,6 +18,72 @@ cuDF jar, that is either preinstalled in the Spark classpath on all nodes or sub that uses the RAPIDS Accelerator For Apache Spark. See the [getting-started guide](https://nvidia.github.io/spark-rapids/Getting-Started/) for more details. +## Release v21.12.0 +Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Architecture: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ + + Apache Spark 3.0.1, 3.0.2, 3.0.3, 3.1.1, 3.1.2, 3.2.0, Cloudera CDP 7.1.6, 7.1.7, Databricks 7.3 ML LTS or 9.1 ML LTS Runtime and GCP Dataproc 2.0 + + Apache Hadoop 2.10+ or 3.1.1+ (3.1.1 for nvidia-docker version 2) + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +### Download v21.12.0 +* Download the [RAPIDS + Accelerator for Apache Spark 21.12.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.12.0/rapids-4-spark_2.12-21.12.0.jar) +* Download the [RAPIDS cuDF 21.12.0 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.12.0/cudf-21.12.0-cuda11.jar) + +This package is built against CUDA 11.5 and has [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html) enabled. It is tested +on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Verify signature +* Download the [RAPIDS Accelerator for Apache Spark 21.12.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.12.0/rapids-4-spark_2.12-21.12.0.jar) + and [RAPIDS Accelerator for Apache Spark 21.12.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.12.0/rapids-4-spark_2.12-21.12.0.jar.asc) +* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature: `gpg --verify rapids-4-spark_2.12-21.12.0.jar.asc rapids-4-spark_2.12-21.12.0.jar` + +The output if signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +New functionality and performance improvements for this release include: +* Support decimal precision up to 38 digits (128-bits) +* Support stddev on double in window context +* Support CPU row-based UDF +* CreateArray outputs array of struct +* collect_set outputs array of struct +* ORC reader and writer support for decimal precision up to 38 digits (128-bits) +* ORC writer supports array, map, and struct +* Support SampleExec, rlike +* regexp_replace supports more patterns such as replacing null +* ParquetCachedBatchSerializer supports map +* Add function explainPotentialGpuPlan to print GPU query plan in a CPU Spark cluster +* Spark Qualification Tool + * Detect RDD APIs and JDBC Scan +* Spark Profiling Tool + * Catch OOM errors and log a hint to increase java heap size + * Print potential problems + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + ## Release v21.10.0 Hardware Requirements: From 7dfbe871d13662a6403ed00063429c55529ed735 Mon Sep 17 00:00:00 2001 From: Rong Ou Date: Mon, 6 Dec 2021 16:50:30 -0800 Subject: [PATCH 29/52] Bump up GPU reserve memory to 640MB (#4308) Signed-off-by: Rong Ou --- docs/configs.md | 2 +- .../src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index 70d9af409db..a22250d37f3 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -41,7 +41,7 @@ Name | Description | Default Value spark.rapids.memory.gpu.oomDumpDir|The path to a local directory where a heap dump will be created if the GPU encounters an unrecoverable out-of-memory (OOM) error. The filename will be of the form: "gpu-oom-.hprof" where is the process ID.|None spark.rapids.memory.gpu.pool|Select the RMM pooling allocator to use. Valid values are "DEFAULT", "ARENA", "ASYNC", and "NONE". With "DEFAULT", the RMM pool allocator is used; with "ARENA", the RMM arena allocator is used; with "ASYNC", the new CUDA stream-ordered memory allocator in CUDA 11.2+ is used. If set to "NONE", pooling is disabled and RMM just passes through to CUDA memory allocation directly. Note: "ARENA" is the recommended pool allocator if CUDF is built with Per-Thread Default Stream (PTDS), as "DEFAULT" is known to be unstable (https://github.com/NVIDIA/spark-rapids/issues/1141)|ARENA spark.rapids.memory.gpu.pooling.enabled|Should RMM act as a pooling allocator for GPU memory, or should it just pass through to CUDA memory allocation directly. DEPRECATED: please use spark.rapids.memory.gpu.pool instead.|true -spark.rapids.memory.gpu.reserve|The amount of GPU memory that should remain unallocated by RMM and left for system use such as memory needed for kernels and kernel launches.|268435456 +spark.rapids.memory.gpu.reserve|The amount of GPU memory that should remain unallocated by RMM and left for system use such as memory needed for kernels and kernel launches.|671088640 spark.rapids.memory.gpu.unspill.enabled|When a spilled GPU buffer is needed again, should it be unspilled, or only copied back into GPU memory temporarily. Unspilling may be useful for GPU buffers that are needed frequently, for example, broadcast variables; however, it may also increase GPU memory usage|false spark.rapids.memory.host.spillStorageSize|Amount of off-heap host memory to use for buffering spilled GPU data before spilling to local disk|1073741824 spark.rapids.memory.pinnedPool.size|The size of the pinned memory pool in bytes unless otherwise specified. Use 0 to disable the pool.|0 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 ee387d88bbe..056c973226f 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 @@ -352,7 +352,7 @@ object RapidsConf { .doc("The amount of GPU memory that should remain unallocated by RMM and left for " + "system use such as memory needed for kernels and kernel launches.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(ByteUnit.MiB.toBytes(256)) + .createWithDefault(ByteUnit.MiB.toBytes(640)) val HOST_SPILL_STORAGE_SIZE = conf("spark.rapids.memory.host.spillStorageSize") .doc("Amount of off-heap host memory to use for buffering spilled GPU data " + From d03e51d6a0b5b677b01939885193470eb04fc25b Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Mon, 6 Dec 2021 17:08:29 -0800 Subject: [PATCH 30/52] Sanitize column names in ParquetCachedBatchSerializer before writing to Parquet [databricks] (#4258) * Sanitize col names Signed-off-by: Raza Jafri * addressed review comments Signed-off-by: Raza Jafri * addressed review comments Signed-off-by: Raza Jafri Co-authored-by: Raza Jafri --- .../shims/v2/ParquetCachedBatchSerializer.scala | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/rapids/shims/v2/ParquetCachedBatchSerializer.scala b/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/rapids/shims/v2/ParquetCachedBatchSerializer.scala index f526023d33f..bb759f577cb 100644 --- a/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/rapids/shims/v2/ParquetCachedBatchSerializer.scala +++ b/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/rapids/shims/v2/ParquetCachedBatchSerializer.scala @@ -565,11 +565,14 @@ class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer with Arm { }) cbRdd.mapPartitions(iter => CloseableColumnBatchIterator(iter)) } else { + val origSelectedAttributesWithUnambiguousNames = + sanitizeColumnNames(newSelectedAttributes, selectedSchemaWithNames) val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs) input.mapPartitions { cbIter => { new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames, - cacheAttributes, newSelectedAttributes, broadcastedConf).getColumnBatchIterator + cacheAttributes, origSelectedAttributesWithUnambiguousNames, broadcastedConf) + .getColumnBatchIterator } } } @@ -592,11 +595,12 @@ class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer with Arm { conf: SQLConf): RDD[InternalRow] = { val (cachedSchemaWithNames, selectedSchemaWithNames) = getSupportedSchemaFromUnsupported(cacheAttributes, selectedAttributes) + val newSelectedAttributes = sanitizeColumnNames(selectedAttributes, selectedSchemaWithNames) val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs) input.mapPartitions { cbIter => { new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames, - cacheAttributes, selectedAttributes, broadcastedConf).getInternalRowIterator + cacheAttributes, newSelectedAttributes, broadcastedConf).getInternalRowIterator } } } @@ -1357,6 +1361,14 @@ class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer with Arm { } } + // We want to change the original schema to have the new names as well + private def sanitizeColumnNames(originalSchema: Seq[Attribute], + schemaToCopyNamesFrom: Seq[Attribute]): Seq[Attribute] = { + originalSchema.zip(schemaToCopyNamesFrom).map { + case (origAttribute, newAttribute) => origAttribute.withName(newAttribute.name) + } + } + private def getSupportedSchemaFromUnsupported( cachedAttributes: Seq[Attribute], requestedAttributes: Seq[Attribute] = Seq.empty): (Seq[Attribute], Seq[Attribute]) = { From 4b6fb6177aecb420b81438e6fd10f4dfd6ffa085 Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Tue, 7 Dec 2021 10:34:16 +0800 Subject: [PATCH 31/52] Update changelog 21.12 to latest [skip ci] (#4245) * Update changelog 21.12 to latest [skip ci] Signed-off-by: Tim Liu * update the changelog to latest * Update changelog to latest Signed-off-by: Tim Liu --- CHANGELOG.md | 280 ++++++++++++++++++++++++++++++++++++- scripts/generate-changelog | 8 +- 2 files changed, 282 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index efc3a259509..25c4d2a7d30 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,280 @@ # Change log -Generated on 2021-10-28 +Generated on 2021-12-07 + +## Release 21.12 + +### Features +||| +|:---|:---| +|[#3953](https://github.com/NVIDIA/spark-rapids/issues/3953)|[FEA] Audit: Add array support to union by name | +|[#4085](https://github.com/NVIDIA/spark-rapids/issues/4085)|[FEA] Decimal 128 Support: Concat| +|[#4073](https://github.com/NVIDIA/spark-rapids/issues/4073)|[FEA] Decimal 128 Support: MapKeys, MapValues, MapEntries| +|[#3432](https://github.com/NVIDIA/spark-rapids/issues/3432)|[FEA] Qualification tool checks if there is any "Scan JDBCRelation" and count it as "problematic"| +|[#3824](https://github.com/NVIDIA/spark-rapids/issues/3824)|[FEA] Support MapType in ParquetCachedBatchSerializer| +|[#4048](https://github.com/NVIDIA/spark-rapids/issues/4048)|[FEA] WindowExpression support for Decimal 128 in Spark 320| +|[#4047](https://github.com/NVIDIA/spark-rapids/issues/4047)|[FEA] Literal support for Decimal 128 in Spark 320| +|[#3863](https://github.com/NVIDIA/spark-rapids/issues/3863)|[FEA] Add Spark 3.3.0-SNAPSHOT Shim | +|[#3814](https://github.com/NVIDIA/spark-rapids/issues/3814)|[FEA] stddev stddev_samp and std should be supported over a window| +|[#3370](https://github.com/NVIDIA/spark-rapids/issues/3370)|[FEA] Add support for Databricks 9.1 runtime| +|[#3876](https://github.com/NVIDIA/spark-rapids/issues/3876)|[FEA] Support REGEXP_REPLACE to replace null values| +|[#3784](https://github.com/NVIDIA/spark-rapids/issues/3784)|[FEA] Support ORC write Map column(single level)| +|[#3470](https://github.com/NVIDIA/spark-rapids/issues/3470)|[FEA] Add shims for 3.2.1-SNAPSHOT| +|[#3855](https://github.com/NVIDIA/spark-rapids/issues/3855)|[FEA] CPU based UDF to run efficiently and transfer data back to GPU for supported operations| +|[#3739](https://github.com/NVIDIA/spark-rapids/issues/3739)|[FEA] Provide an explicit config for fallback on CPU if plan rewrite fails| +|[#3888](https://github.com/NVIDIA/spark-rapids/issues/3888)|[FEA] Decimal 128 Support: Add a "Trust me I know it will not overflow config"| +|[#3088](https://github.com/NVIDIA/spark-rapids/issues/3088)|[FEA] Profile tool print problematic operations| +|[#3886](https://github.com/NVIDIA/spark-rapids/issues/3886)|[FEA] Decimal 128 Support: Extend the range for Decimal Multiply and Divide| +|[#79](https://github.com/NVIDIA/spark-rapids/issues/79)|[FEA] Support Size operation| +|[#3880](https://github.com/NVIDIA/spark-rapids/issues/3880)|[FEA] Decimal 128 Support: Average aggregation| +|[#3659](https://github.com/NVIDIA/spark-rapids/issues/3659)|[FEA] External tool integration with Qualification tool| +|[#2](https://github.com/NVIDIA/spark-rapids/issues/2)|[FEA] RLIKE support| +|[#3192](https://github.com/NVIDIA/spark-rapids/issues/3192)|[FEA] Support decimal type in ORC writer| +|[#3419](https://github.com/NVIDIA/spark-rapids/issues/3419)|[FEA] Add support for org.apache.spark.sql.execution.SampleExec| +|[#3535](https://github.com/NVIDIA/spark-rapids/issues/3535)|[FEA] Qualification tool can detect RDD APIs in SQL plan| +|[#3494](https://github.com/NVIDIA/spark-rapids/issues/3494)|[FEA] Support structs in ORC writer| +|[#3514](https://github.com/NVIDIA/spark-rapids/issues/3514)|[FEA] Support collect_set on struct in aggregation context| +|[#3515](https://github.com/NVIDIA/spark-rapids/issues/3515)|[FEA] Support CreateArray to produce array(struct)| +|[#3116](https://github.com/NVIDIA/spark-rapids/issues/3116)|[FEA] Support Maps, Lists, and Structs as non-key columns on joins| +|[#2054](https://github.com/NVIDIA/spark-rapids/issues/2054)|[FEA] Add support for Arrays to ParquetCachedBatchSerializer| +|[#3573](https://github.com/NVIDIA/spark-rapids/issues/3573)|[FEA] Support Cache(PCBS) Array-of-Struct| + +### Performance +||| +|:---|:---| +|[#3768](https://github.com/NVIDIA/spark-rapids/issues/3768)|[DOC] document databricks init script required for UCX| +|[#2867](https://github.com/NVIDIA/spark-rapids/issues/2867)|[FEA] Make LZ4_CHUNK_SIZE configurable| +|[#3832](https://github.com/NVIDIA/spark-rapids/issues/3832)|[FEA] AST enabled GpuBroadcastNestedLoopJoin left side can't be small| +|[#3798](https://github.com/NVIDIA/spark-rapids/issues/3798)|[FEA] bounds checking in joins can be expensive| +|[#3603](https://github.com/NVIDIA/spark-rapids/issues/3603)|[FEA] Allocate UCX bounce buffers outside of RMM if ASYNC allocator is enabled| + +### Bugs Fixed +||| +|:---|:---| +|[#4253](https://github.com/NVIDIA/spark-rapids/issues/4253)|[BUG] Dependencies missing of spark-rapids v21.12.0 release jars| +|[#4216](https://github.com/NVIDIA/spark-rapids/issues/4216)|[BUG] AQE Crashing Spark RAPIDS when using filter() and union()| +|[#4188](https://github.com/NVIDIA/spark-rapids/issues/4188)|[BUG] data corruption in GpuBroadcastNestedLoopJoin with empty relations edge case| +|[#4191](https://github.com/NVIDIA/spark-rapids/issues/4191)|[BUG] failed to read DECIMAL128 within MapType from ORC| +|[#4175](https://github.com/NVIDIA/spark-rapids/issues/4175)|[BUG] arithmetic_ops_test failed in spark 3.2.0| +|[#4162](https://github.com/NVIDIA/spark-rapids/issues/4162)|[BUG] isCastDecimalToStringEnabled is never called| +|[#3894](https://github.com/NVIDIA/spark-rapids/issues/3894)|[BUG] test_pandas_scalar_udf and test_pandas_map_udf failed in UCX standalone CI run| +|[#3970](https://github.com/NVIDIA/spark-rapids/issues/3970)|[BUG] mismatching timezone settings on executor and driver can cause ORC read data corruption| +|[#4141](https://github.com/NVIDIA/spark-rapids/issues/4141)|[BUG] Unable to start the RapidsShuffleManager in databricks 9.1| +|[#4102](https://github.com/NVIDIA/spark-rapids/issues/4102)|[BUG] udf-example build failed: Unknown CMake command "cpm_check_if_package_already_added".| +|[#4084](https://github.com/NVIDIA/spark-rapids/issues/4084)|[BUG] window on unbounded preceeding and unbounded following can produce incorrect results.| +|[#3990](https://github.com/NVIDIA/spark-rapids/issues/3990)|[BUG] Scaladoc link warnings in ParquetCachedBatchSerializer and ExplainPlan| +|[#4108](https://github.com/NVIDIA/spark-rapids/issues/4108)|[BUG] premerge fails due to Spark 3.3.0 HadoopFsRelation after SPARK-37289| +|[#4042](https://github.com/NVIDIA/spark-rapids/issues/4042)|[BUG] cudf_udf tests fail on nightly Integration test run| +|[#3743](https://github.com/NVIDIA/spark-rapids/issues/3743)|[BUG] Implicitly catching all exceptions warning in GpuOverrides| +|[#4069](https://github.com/NVIDIA/spark-rapids/issues/4069)|[BUG] parquet_test.py pytests FAILED on Databricks-9.1-ML-spark-3.1.2| +|[#3461](https://github.com/NVIDIA/spark-rapids/issues/3461)|[BUG] Cannot build project from a sub-directory| +|[#4053](https://github.com/NVIDIA/spark-rapids/issues/4053)|[BUG] buildall uses a stale aggregator dependency during test compilation| +|[#3703](https://github.com/NVIDIA/spark-rapids/issues/3703)|[BUG] test_hash_groupby_approx_percentile_long_repeated_keys failed with TypeError| +|[#3706](https://github.com/NVIDIA/spark-rapids/issues/3706)|[BUG] approx_percentile returns array of zero percentiles instead of null in some cases| +|[#4017](https://github.com/NVIDIA/spark-rapids/issues/4017)|[BUG] Why is the hash aggregate not handling empty result expressions| +|[#3994](https://github.com/NVIDIA/spark-rapids/issues/3994)|[BUG] can't open notebook 'docs/demo/GCP/mortgage-xgboost4j-gpu-scala.ipynb'| +|[#3996](https://github.com/NVIDIA/spark-rapids/issues/3996)|[BUG] Exception happened when getting a null row| +|[#3999](https://github.com/NVIDIA/spark-rapids/issues/3999)|[BUG] Integration cache_test failures - ArrayIndexOutOfBoundsException| +|[#3532](https://github.com/NVIDIA/spark-rapids/issues/3532)|[BUG] DatabricksShimVersion must carry runtime version info| +|[#3834](https://github.com/NVIDIA/spark-rapids/issues/3834)|[BUG] Approx_percentile deserialize error when calling "show" rather than "collect"| +|[#3992](https://github.com/NVIDIA/spark-rapids/issues/3992)|[BUG] failed create-parallel-world in databricks build| +|[#3987](https://github.com/NVIDIA/spark-rapids/issues/3987)|[BUG] "mvn clean package -DskipTests" is no longer working| +|[#3866](https://github.com/NVIDIA/spark-rapids/issues/3866)|[BUG] RLike integration tests failing on Azure Databricks 7.3| +|[#3980](https://github.com/NVIDIA/spark-rapids/issues/3980)|[BUG] udf-example build failed due to maven-antrun-plugin upgrade| +|[#3966](https://github.com/NVIDIA/spark-rapids/issues/3966)|[BUG] udf-examples module fails on `mvn compile` and `mvn test`| +|[#3977](https://github.com/NVIDIA/spark-rapids/issues/3977)|[BUG] databricks aggregator jar deployed failed| +|[#3915](https://github.com/NVIDIA/spark-rapids/issues/3915)|[BUG] typo in verify_same_sha_for_unshimmed prevents the offending class file name from being logged. | +|[#1304](https://github.com/NVIDIA/spark-rapids/issues/1304)|[BUG] Query fails with HostColumnarToGpu doesn't support Structs| +|[#3924](https://github.com/NVIDIA/spark-rapids/issues/3924)|[BUG] ExpressionEncoder does not work for input in `GpuScalaUDF` | +|[#3911](https://github.com/NVIDIA/spark-rapids/issues/3911)|[BUG] CI fails on an inconsistent set of partial builds| +|[#2896](https://github.com/NVIDIA/spark-rapids/issues/2896)|[BUG] Extra GpuColumnarToRow when using ParquetCachedBatchSerializer on databricks| +|[#3864](https://github.com/NVIDIA/spark-rapids/issues/3864)|[BUG] test_sample_produce_empty_batch failed in dataproc| +|[#3823](https://github.com/NVIDIA/spark-rapids/issues/3823)|[BUG]binary-dedup.sh script fails on mac| +|[#3658](https://github.com/NVIDIA/spark-rapids/issues/3658)|[BUG] DataFrame actions failing with error: Error : java.lang.NoClassDefFoundError: Could not initialize class com.nvidia.spark.rapids.GpuOverrides withlatest 21.10 jars| +|[#3857](https://github.com/NVIDIA/spark-rapids/issues/3857)|[BUG] nightly build push dist packge w/ single version of spark| +|[#3854](https://github.com/NVIDIA/spark-rapids/issues/3854)|[BUG] not found: type PoissonDistribution in databricks build| +|[#3852](https://github.com/NVIDIA/spark-rapids/issues/3852)|spark-nightly-build deploys all modules due to typo in `-pl`| +|[#3844](https://github.com/NVIDIA/spark-rapids/issues/3844)|[BUG] nightly spark311cdh build failed| +|[#3843](https://github.com/NVIDIA/spark-rapids/issues/3843)|[BUG] databricks nightly deploy failed| +|[#3705](https://github.com/NVIDIA/spark-rapids/issues/3705)|[BUG] Change `nullOnDivideByZero` from runtime parameter to aggregate expression for `stddev` and `variance` aggregation families| +|[#3614](https://github.com/NVIDIA/spark-rapids/issues/3614)|[BUG] ParquetMaterializer.scala appears in both v1 and v2 shims| +|[#3430](https://github.com/NVIDIA/spark-rapids/issues/3430)|[BUG] Profiling tool silently stops without producing any output on a Synapse Spark event log| +|[#3311](https://github.com/NVIDIA/spark-rapids/issues/3311)|[BUG] cache_test.py failed w/ cache.serializer in spark 3.1.2| +|[#3710](https://github.com/NVIDIA/spark-rapids/issues/3710)|[BUG] Usage of Class.forName without specifying a classloader| +|[#3462](https://github.com/NVIDIA/spark-rapids/issues/3462)|[BUG] IDE complains about duplicate ShimBasePythonRunner instances| +|[#3476](https://github.com/NVIDIA/spark-rapids/issues/3476)|[BUG] test_non_empty_ctas fails on yarn| + +### PRs +||| +|:---|:---| +|[#4258](https://github.com/NVIDIA/spark-rapids/pull/4258)|Sanitize column names in ParquetCachedBatchSerializer before writing to Parquet| +|[#4308](https://github.com/NVIDIA/spark-rapids/pull/4308)|Bump up GPU reserve memory to 640MB| +|[#4307](https://github.com/NVIDIA/spark-rapids/pull/4307)|Update Download page for 21.12 [skip ci]| +|[#4261](https://github.com/NVIDIA/spark-rapids/pull/4261)|Update cudfjni version to released 21.12.0| +|[#4265](https://github.com/NVIDIA/spark-rapids/pull/4265)|Remove aggregator dependency before deploying dist artifact| +|[#4030](https://github.com/NVIDIA/spark-rapids/pull/4030)|Support code coverage report with single version jar [skip ci]| +|[#4287](https://github.com/NVIDIA/spark-rapids/pull/4287)|Update 21.12 compatibility guide for known regexp issue [skip ci]| +|[#4242](https://github.com/NVIDIA/spark-rapids/pull/4242)|Fix indentation issue in getting-started-k8s guide [skip ci]| +|[#4263](https://github.com/NVIDIA/spark-rapids/pull/4263)|Add missing ORC write tests on Map of Decimal| +|[#4257](https://github.com/NVIDIA/spark-rapids/pull/4257)|Implement getShuffleRDD and fixup mismatched output types on shuffle reuse| +|[#4250](https://github.com/NVIDIA/spark-rapids/pull/4250)|Update the release script [skip ci]| +|[#4222](https://github.com/NVIDIA/spark-rapids/pull/4222)|Add arguments support to 'databricks/run-tests.py'| +|[#4233](https://github.com/NVIDIA/spark-rapids/pull/4233)|Add databricks init script for UCX| +|[#4231](https://github.com/NVIDIA/spark-rapids/pull/4231)|RAPIDS Shuffle Manager fallback if security is enabled| +|[#4228](https://github.com/NVIDIA/spark-rapids/pull/4228)|Fix unconditional nested loop joins on empty tables| +|[#4217](https://github.com/NVIDIA/spark-rapids/pull/4217)|Enable event log for qualification & profiling tools testing from IT| +|[#4202](https://github.com/NVIDIA/spark-rapids/pull/4202)|Parameter for the Databricks zone-id [skip ci]| +|[#4199](https://github.com/NVIDIA/spark-rapids/pull/4199)|modify some words for synapse getting started guide[skip ci]| +|[#4200](https://github.com/NVIDIA/spark-rapids/pull/4200)|Disable approx percentile tests that intermittently fail| +|[#4187](https://github.com/NVIDIA/spark-rapids/pull/4187)|Added a getting started guide for Synapse[skip ci]| +|[#4192](https://github.com/NVIDIA/spark-rapids/pull/4192)|Fix ORC read DECIMAL128 inside MapType| +|[#4173](https://github.com/NVIDIA/spark-rapids/pull/4173)|Update approx percentile docs to link to issue 4060 [skip ci]| +|[#4174](https://github.com/NVIDIA/spark-rapids/pull/4174)|Document Bloop, Metals and VS code as an IDE option [skip ci]| +|[#4181](https://github.com/NVIDIA/spark-rapids/pull/4181)|Fix element_at for 3.2.0 and array/struct cast| +|[#4110](https://github.com/NVIDIA/spark-rapids/pull/4110)|Add a getting started guide on workload qualification [skip ci]| +|[#4106](https://github.com/NVIDIA/spark-rapids/pull/4106)|Add docs for MIG on YARN [skip ci]| +|[#4100](https://github.com/NVIDIA/spark-rapids/pull/4100)|Add PCA example to ml-integration page [skip ci]| +|[#4177](https://github.com/NVIDIA/spark-rapids/pull/4177)|Decimal128: added missing decimal128 signature on Spark 32X| +|[#4161](https://github.com/NVIDIA/spark-rapids/pull/4161)|More integration tests with decimal128| +|[#4165](https://github.com/NVIDIA/spark-rapids/pull/4165)|Fix type checks for get array item in 3.2.0| +|[#4163](https://github.com/NVIDIA/spark-rapids/pull/4163)|Enable config to check for casting decimals to strings| +|[#4154](https://github.com/NVIDIA/spark-rapids/pull/4154)|Use num_slices to guarantee partition shape in the pandas udf tests| +|[#4129](https://github.com/NVIDIA/spark-rapids/pull/4129)|Check executor timezone is same as driver timezone when running on GPU| +|[#4139](https://github.com/NVIDIA/spark-rapids/pull/4139)|Decimal128 Support| +|[#4128](https://github.com/NVIDIA/spark-rapids/pull/4128)|Fix build errors in udf-examples native build| +|[#4063](https://github.com/NVIDIA/spark-rapids/pull/4063)|Regexp_replace support regexp| +|[#4125](https://github.com/NVIDIA/spark-rapids/pull/4125)|Remove unused imports| +|[#4052](https://github.com/NVIDIA/spark-rapids/pull/4052)|Support null safe host column vector| +|[#4116](https://github.com/NVIDIA/spark-rapids/pull/4116)|Add in tests to check for overflow in unbounded window| +|[#4111](https://github.com/NVIDIA/spark-rapids/pull/4111)|Added external doc links for JRE and Spark| +|[#4105](https://github.com/NVIDIA/spark-rapids/pull/4105)|Enforce checks for unused imports and missed interpolation| +|[#4107](https://github.com/NVIDIA/spark-rapids/pull/4107)|Set the task context in background reader threads| +|[#4114](https://github.com/NVIDIA/spark-rapids/pull/4114)|Refactoring cudf_udf test setup| +|[#4109](https://github.com/NVIDIA/spark-rapids/pull/4109)|Stop using redundant partitionSchemaOption dropped in 3.3.0| +|[#4097](https://github.com/NVIDIA/spark-rapids/pull/4097)|Enable auto-merge from branch-21.12 to branch-22.02 [skip ci]| +|[#4094](https://github.com/NVIDIA/spark-rapids/pull/4094)|Remove spark311db shim layer| +|[#4082](https://github.com/NVIDIA/spark-rapids/pull/4082)|Add abfs and abfss to the cloud scheme| +|[#4071](https://github.com/NVIDIA/spark-rapids/pull/4071)|Treat scalac warnings as errors| +|[#4043](https://github.com/NVIDIA/spark-rapids/pull/4043)|Promote cudf as dist direct dependency, mark aggregator provided| +|[#4076](https://github.com/NVIDIA/spark-rapids/pull/4076)|Sets the GPU device id in the UCX early start thread| +|[#4087](https://github.com/NVIDIA/spark-rapids/pull/4087)|Regex parser improvements and bug fixes| +|[#4079](https://github.com/NVIDIA/spark-rapids/pull/4079)|verify "Add array support to union by name " by adding an integration test| +|[#4090](https://github.com/NVIDIA/spark-rapids/pull/4090)|Update pre-merge expression for 2022+ CI [skip ci]| +|[#4049](https://github.com/NVIDIA/spark-rapids/pull/4049)|Change Databricks image from 8.2 to 9.1 [skip ci]| +|[#4051](https://github.com/NVIDIA/spark-rapids/pull/4051)|Upgrade ORC version from 1.5.8 to 1.5.10| +|[#4080](https://github.com/NVIDIA/spark-rapids/pull/4080)|Add case insensitive when clipping parquet blocks| +|[#4083](https://github.com/NVIDIA/spark-rapids/pull/4083)|Fix compiler warning in regex transpiler| +|[#4070](https://github.com/NVIDIA/spark-rapids/pull/4070)|Support building from sub directory| +|[#4072](https://github.com/NVIDIA/spark-rapids/pull/4072)|Fix overflow checking on optimized decimal sum| +|[#4067](https://github.com/NVIDIA/spark-rapids/pull/4067)|Append new authorized user to blossom-ci whitelist [skip ci]| +|[#4066](https://github.com/NVIDIA/spark-rapids/pull/4066)|Temply disable cudf_udf test| +|[#4057](https://github.com/NVIDIA/spark-rapids/pull/4057)|Restore original ASL 2.0 license text| +|[#3937](https://github.com/NVIDIA/spark-rapids/pull/3937)|Qualification tool: Detect JDBCRelation in eventlog| +|[#3925](https://github.com/NVIDIA/spark-rapids/pull/3925)|verify AQE and DPP both on| +|[#3982](https://github.com/NVIDIA/spark-rapids/pull/3982)|Fix the issue of parquet reading with case insensitive schema| +|[#4054](https://github.com/NVIDIA/spark-rapids/pull/4054)|Use install for the base version build thread [skip ci]| +|[#4008](https://github.com/NVIDIA/spark-rapids/pull/4008)|[Doc] Update the getting started guide for databricks: Change from 8.2 to 9.1 runtime [skip ci]| +|[#4010](https://github.com/NVIDIA/spark-rapids/pull/4010)|Enable MapType for ParquetCachedBatchSerializer| +|[#4046](https://github.com/NVIDIA/spark-rapids/pull/4046)|lower GPU memory reserve to 256MB| +|[#3770](https://github.com/NVIDIA/spark-rapids/pull/3770)|Enable approx percentile tests| +|[#4038](https://github.com/NVIDIA/spark-rapids/pull/4038)|Change the `catalystConverter` to be a Scala `val`.| +|[#4035](https://github.com/NVIDIA/spark-rapids/pull/4035)|Hash aggregate fix empty resultExpressions| +|[#3998](https://github.com/NVIDIA/spark-rapids/pull/3998)|Check for CPU cores and free memory in IT script| +|[#3984](https://github.com/NVIDIA/spark-rapids/pull/3984)|Check for data write command before inserting hash sort optimization| +|[#4019](https://github.com/NVIDIA/spark-rapids/pull/4019)|initialize RMM with a single pool size| +|[#3993](https://github.com/NVIDIA/spark-rapids/pull/3993)|Qualification tool: Remove "unsupported" word for nested complex types| +|[#4033](https://github.com/NVIDIA/spark-rapids/pull/4033)|skip spark 330 tests temporarily in nightly [skip ci]| +|[#4029](https://github.com/NVIDIA/spark-rapids/pull/4029)|Update buildall script and the build doc [skip ci]| +|[#4014](https://github.com/NVIDIA/spark-rapids/pull/4014)|fix can't open notebook 'docs/demo/GCP/mortgage-xgboost4j-gpu-scala.ipynb'[skip ci]| +|[#4024](https://github.com/NVIDIA/spark-rapids/pull/4024)|Allow using a custom Spark Resource Name for a GPU| +|[#4012](https://github.com/NVIDIA/spark-rapids/pull/4012)|Add Apache Spark 3.3.0-SNAPSHOT Shims| +|[#4021](https://github.com/NVIDIA/spark-rapids/pull/4021)|Explicitly use the public version of ParquetCachedBatchSerializer| +|[#3869](https://github.com/NVIDIA/spark-rapids/pull/3869)|Add Std dev samp for windowing| +|[#3960](https://github.com/NVIDIA/spark-rapids/pull/3960)|Use a fixed RMM pool size| +|[#3767](https://github.com/NVIDIA/spark-rapids/pull/3767)|Add shim for Databricks 9.1| +|[#3862](https://github.com/NVIDIA/spark-rapids/pull/3862)|Prevent approx_percentile aggregate from being split between CPU and GPU| +|[#3871](https://github.com/NVIDIA/spark-rapids/pull/3871)|Add integration test for RLike with embedded null in input| +|[#3968](https://github.com/NVIDIA/spark-rapids/pull/3968)|Allow null character in regexp_replace pattern| +|[#3821](https://github.com/NVIDIA/spark-rapids/pull/3821)|Support ORC write Map column| +|[#3991](https://github.com/NVIDIA/spark-rapids/pull/3991)|Fix aggregator jar copy logic| +|[#3973](https://github.com/NVIDIA/spark-rapids/pull/3973)|Add shims for Apache Spark 3.2.1-SNAPSHOT builds| +|[#3967](https://github.com/NVIDIA/spark-rapids/pull/3967)|Bring back AST support for BNLJ inner joins| +|[#3947](https://github.com/NVIDIA/spark-rapids/pull/3947)|Enable rlike tests on databricks| +|[#3981](https://github.com/NVIDIA/spark-rapids/pull/3981)|Replace tasks w/ target of maven-antrun-plugin in udf-example| +|[#3976](https://github.com/NVIDIA/spark-rapids/pull/3976)|Replace long artifact lists with an ant loop| +|[#3972](https://github.com/NVIDIA/spark-rapids/pull/3972)|Revert udf-examples dependency change to restore test build phase| +|[#3978](https://github.com/NVIDIA/spark-rapids/pull/3978)|Update aggregator jar name in databricks deploy script| +|[#3965](https://github.com/NVIDIA/spark-rapids/pull/3965)|Add how-to resolve auto-merge conflict [skip ci]| +|[#3963](https://github.com/NVIDIA/spark-rapids/pull/3963)|Add a dedicated RapidsConf option to tolerate GpuOverrides apply failures| +|[#3923](https://github.com/NVIDIA/spark-rapids/pull/3923)|Prepare for 3.2.1 shim, various shim build fixes and improvements| +|[#3969](https://github.com/NVIDIA/spark-rapids/pull/3969)|add doc on using compute-sanitizer| +|[#3964](https://github.com/NVIDIA/spark-rapids/pull/3964)|Qualification tool: Catch exception for invalid regex patterns| +|[#3961](https://github.com/NVIDIA/spark-rapids/pull/3961)|Avoid using HostColumnarToGpu for nested types| +|[#3910](https://github.com/NVIDIA/spark-rapids/pull/3910)|Refactor the aggregate API| +|[#3897](https://github.com/NVIDIA/spark-rapids/pull/3897)|Support running CPU based UDF efficiently| +|[#3950](https://github.com/NVIDIA/spark-rapids/pull/3950)|Fix failed auto-merge #3939| +|[#3946](https://github.com/NVIDIA/spark-rapids/pull/3946)|Document compatability of operations with side effects.| +|[#3945](https://github.com/NVIDIA/spark-rapids/pull/3945)|Update udf-examples dependencies to use dist jar| +|[#3938](https://github.com/NVIDIA/spark-rapids/pull/3938)|remove GDS alignment code| +|[#3943](https://github.com/NVIDIA/spark-rapids/pull/3943)|Add artifact revisions check for nightly tests [skip ci]| +|[#3933](https://github.com/NVIDIA/spark-rapids/pull/3933)|Profiling tool: Print potential problems| +|[#3926](https://github.com/NVIDIA/spark-rapids/pull/3926)|Add zip unzip to integration tests dockerfiles [skip ci]| +|[#3757](https://github.com/NVIDIA/spark-rapids/pull/3757)|Update to nvcomp-2.x JNI APIs| +|[#3922](https://github.com/NVIDIA/spark-rapids/pull/3922)|Stop using -U in build merges aggregator jars of nightly [skip ci]| +|[#3907](https://github.com/NVIDIA/spark-rapids/pull/3907)|Add version properties to integration tests modules| +|[#3912](https://github.com/NVIDIA/spark-rapids/pull/3912)|Stop using -U in the build that merges all aggregator jars| +|[#3909](https://github.com/NVIDIA/spark-rapids/pull/3909)|Fix warning when catching all throwables in GpuOverrides| +|[#3766](https://github.com/NVIDIA/spark-rapids/pull/3766)|Use JCudfSerialization to deserialize a table to host columns| +|[#3820](https://github.com/NVIDIA/spark-rapids/pull/3820)|Advertise CPU orderingSatisfies| +|[#3858](https://github.com/NVIDIA/spark-rapids/pull/3858)|update emr 6.4 getting started doc and pic[skip ci]| +|[#3899](https://github.com/NVIDIA/spark-rapids/pull/3899)|Fix sample test cases| +|[#3896](https://github.com/NVIDIA/spark-rapids/pull/3896)|Xfail the sample tests temporarily| +|[#3848](https://github.com/NVIDIA/spark-rapids/pull/3848)|Fix binary-dedupe failures and improve its performance on macOS| +|[#3867](https://github.com/NVIDIA/spark-rapids/pull/3867)|Disable rlike integration tests on Databricks| +|[#3850](https://github.com/NVIDIA/spark-rapids/pull/3850)|Add explain Plugin API for CPU plan| +|[#3868](https://github.com/NVIDIA/spark-rapids/pull/3868)|Fix incorrect schema of nested types of union - audit SPARK-36673| +|[#3860](https://github.com/NVIDIA/spark-rapids/pull/3860)|Add unit test for GpuKryoRegistrator| +|[#3847](https://github.com/NVIDIA/spark-rapids/pull/3847)|Add Running Qualification App API| +|[#3861](https://github.com/NVIDIA/spark-rapids/pull/3861)|Revert "Fix typo in nightly deploy project list (#3853)" [skip ci]| +|[#3796](https://github.com/NVIDIA/spark-rapids/pull/3796)|Add Rlike support| +|[#3856](https://github.com/NVIDIA/spark-rapids/pull/3856)|Fix not found: type PoissonDistribution in databricks build| +|[#3853](https://github.com/NVIDIA/spark-rapids/pull/3853)|Fix typo in nightly deploy project list| +|[#3831](https://github.com/NVIDIA/spark-rapids/pull/3831)|Support decimal type in ORC writer| +|[#3789](https://github.com/NVIDIA/spark-rapids/pull/3789)|GPU sample exec| +|[#3846](https://github.com/NVIDIA/spark-rapids/pull/3846)|Include pluginRepository for cdh build| +|[#3819](https://github.com/NVIDIA/spark-rapids/pull/3819)|Qualification tool: Detect RDD Api's in SQL plan| +|[#3835](https://github.com/NVIDIA/spark-rapids/pull/3835)|Minor cleanup: do not set cuda stream to null| +|[#3845](https://github.com/NVIDIA/spark-rapids/pull/3845)|Include 'DB_SHIM_NAME' from Databricks jar path to fix nightly deploy [skip ci]| +|[#3523](https://github.com/NVIDIA/spark-rapids/pull/3523)|Interpolate spark.version.classifier in build.dir| +|[#3813](https://github.com/NVIDIA/spark-rapids/pull/3813)|Change `nullOnDivideByZero` from runtime parameter to aggregate expression for `stddev` and `variance` aggregations| +|[#3791](https://github.com/NVIDIA/spark-rapids/pull/3791)|Add audit script to get list of commits from Apache Spark master branch| +|[#3744](https://github.com/NVIDIA/spark-rapids/pull/3744)|Add developer documentation for setting up Microk8s [skip ci]| +|[#3817](https://github.com/NVIDIA/spark-rapids/pull/3817)|Fix auto-merge conflict 3816 [skip ci]| +|[#3804](https://github.com/NVIDIA/spark-rapids/pull/3804)|Missing statistics in GpuBroadcastNestedLoopJoin| +|[#3799](https://github.com/NVIDIA/spark-rapids/pull/3799)|Optimize out bounds checking for joins when the gather map has only valid entries| +|[#3801](https://github.com/NVIDIA/spark-rapids/pull/3801)|Update premerge to use the combined snapshots jar | +|[#3696](https://github.com/NVIDIA/spark-rapids/pull/3696)|Support nested types in ORC writer| +|[#3790](https://github.com/NVIDIA/spark-rapids/pull/3790)|Fix overflow when casting integral to neg scale decimal| +|[#3779](https://github.com/NVIDIA/spark-rapids/pull/3779)|Enable some union of structs tests that were marked xfail| +|[#3787](https://github.com/NVIDIA/spark-rapids/pull/3787)|Fix auto-merge conflict 3786 from branch-21.10 [skip ci]| +|[#3782](https://github.com/NVIDIA/spark-rapids/pull/3782)|Fix auto-merge conflict 3781 [skip ci]| +|[#3778](https://github.com/NVIDIA/spark-rapids/pull/3778)|Remove extra ParquetMaterializer.scala file| +|[#3773](https://github.com/NVIDIA/spark-rapids/pull/3773)|Restore disabled ORC and Parquet tests| +|[#3714](https://github.com/NVIDIA/spark-rapids/pull/3714)|Qualification tool: Error handling while processing large event logs| +|[#3758](https://github.com/NVIDIA/spark-rapids/pull/3758)|Temporarily disable timestamp read tests for Parquet and ORC| +|[#3748](https://github.com/NVIDIA/spark-rapids/pull/3748)|Fix merge conflict with branch-21.10| +|[#3700](https://github.com/NVIDIA/spark-rapids/pull/3700)|CollectSet supports structs| +|[#3740](https://github.com/NVIDIA/spark-rapids/pull/3740)|Throw Exception if failure to load ParquetCachedBatchSerializer class| +|[#3726](https://github.com/NVIDIA/spark-rapids/pull/3726)|Replace Class.forName with ShimLoader.loadClass| +|[#3690](https://github.com/NVIDIA/spark-rapids/pull/3690)|Added support for Array[Struct] to GpuCreateArray| +|[#3728](https://github.com/NVIDIA/spark-rapids/pull/3728)|Qualification tool: Fix bug to process correct listeners| +|[#3734](https://github.com/NVIDIA/spark-rapids/pull/3734)|Fix squashed merge from #3725| +|[#3725](https://github.com/NVIDIA/spark-rapids/pull/3725)|Fix merge conflict with branch-21.10| +|[#3680](https://github.com/NVIDIA/spark-rapids/pull/3680)|cudaMalloc UCX bounce buffers when async allocator is used| +|[#3681](https://github.com/NVIDIA/spark-rapids/pull/3681)|Clean up and document metrics| +|[#3674](https://github.com/NVIDIA/spark-rapids/pull/3674)|Move file TestingV2Source.Scala| +|[#3617](https://github.com/NVIDIA/spark-rapids/pull/3617)|Update Version to 21.12.0-SNAPSHOT| +|[#3612](https://github.com/NVIDIA/spark-rapids/pull/3612)|Add support for nested types as non-key columns on joins | +|[#3619](https://github.com/NVIDIA/spark-rapids/pull/3619)|Added support for Array of Structs| ## Release 21.10 @@ -11,6 +286,7 @@ Generated on 2021-10-28 |[#13](https://github.com/NVIDIA/spark-rapids/issues/13)|[FEA] Percentile support| |[#3606](https://github.com/NVIDIA/spark-rapids/issues/3606)|[FEA] Support approx_percentile on GPU with decimal type| |[#3552](https://github.com/NVIDIA/spark-rapids/issues/3552)|[FEA] extend allowed datatypes for add and multiply in ANSI mode | +|[#3450](https://github.com/NVIDIA/spark-rapids/issues/3450)|[FEA] test the UCX shuffle with the new build changes| |[#3043](https://github.com/NVIDIA/spark-rapids/issues/3043)|[FEA] Qualification tool: Add support to filter specific configuration values| |[#3413](https://github.com/NVIDIA/spark-rapids/issues/3413)|[FEA] Add in support for transform_keys| |[#3297](https://github.com/NVIDIA/spark-rapids/issues/3297)|[FEA] ORC reader supports reading Map columns.| @@ -48,7 +324,6 @@ Generated on 2021-10-28 ||| |:---|:---| |[#3651](https://github.com/NVIDIA/spark-rapids/issues/3651)|[DOC] Point users to UCX 1.11.2| -|[#3450](https://github.com/NVIDIA/spark-rapids/issues/3450)|[FEA] test the UCX shuffle with the new build changes| |[#2370](https://github.com/NVIDIA/spark-rapids/issues/2370)|[FEA] RAPIDS Shuffle Manager enable/disable config| |[#2923](https://github.com/NVIDIA/spark-rapids/issues/2923)|[FEA] Move to dispatched binops instead of JIT binops| @@ -146,6 +421,7 @@ Generated on 2021-10-28 ### PRs ||| |:---|:---| +|[#3940](https://github.com/NVIDIA/spark-rapids/pull/3940)|Update changelog [skip ci]| |[#3930](https://github.com/NVIDIA/spark-rapids/pull/3930)|Dist artifact with provided aggregator dependency| |[#3918](https://github.com/NVIDIA/spark-rapids/pull/3918)|Update changelog [skip ci]| |[#3906](https://github.com/NVIDIA/spark-rapids/pull/3906)|Doc updated for v2110[skip ci]| diff --git a/scripts/generate-changelog b/scripts/generate-changelog index 839fec6f8d2..0c313f6ec5a 100755 --- a/scripts/generate-changelog +++ b/scripts/generate-changelog @@ -44,13 +44,13 @@ Github personal access token: https://github.com/settings/tokens, and make you h Usage: cd spark-rapids/ - # generate changelog for releases 0.1 to 21.10 + # generate changelog for releases 0.1 to 21.12 scripts/generate-changelog --token= \ - --releases=0.1,0.2,0.3,0.4,0.4.1,0.5,21.06,21.06.1,21.06.2,21.08,21.08.1,21.10 + --releases=0.1,0.2,0.3,0.4,0.4.1,0.5,21.06,21.06.1,21.06.2,21.08,21.08.1,21.10,21.12 - # generate changelog for releases 0.1 to 21.10 to /tmp/CHANGELOG.md + # generate changelog for releases 0.1 to 21.12 to /tmp/CHANGELOG.md GITHUB_TOKEN= scripts/generate-changelog \ - --releases=0.1,0.2,0.3,0.4,0.4.1,0.5,21.06,21.06.1,21.06.2,21.08,21.08.1,21.10 \ + --releases=0.1,0.2,0.3,0.4,0.4.1,0.5,21.06,21.06.1,21.06.2,21.08,21.08.1,21.10,21.12 \ --path=/tmp/CHANGELOG.md """ import os From df4a0478301bfd2588c6ad88a79909afedfb60ae Mon Sep 17 00:00:00 2001 From: Allen Xu Date: Tue, 7 Dec 2021 13:19:11 +0800 Subject: [PATCH 32/52] Porting GpuRowToColumnar converters to InternalColumnarRDDConverter (#4206) * Porting GpuRowToColumnar converters to InternalColumnarRDDConverter - allow converting array, map, struct and decimal Signed-off-by: Allen Xu * update but still bugs Signed-off-by: Allen Xu * pass compile Signed-off-by: Allen Xu * bug fix Signed-off-by: Allen Xu * Add in tests for other supported types Signed-off-by: Allen Xu * fix typo * Update sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala Co-authored-by: Jason Lowe * resolve comments * Add license Signed-off-by: Allen Xu * fix typo Signed-off-by: Allen Xu * update * nit fixup Co-authored-by: Allen Xu Co-authored-by: Jason Lowe --- .../InternalColumnarRddConverter.scala | 596 ++++++++++++------ .../spark/rapids/GpuBatchUtilsSuite.scala | 154 +++-- .../InternalColumnarRDDConverterSuite.scala | 276 ++++++++ 3 files changed, 784 insertions(+), 242 deletions(-) create mode 100644 tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala index 8c3af1b23b3..af5a46d3fac 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala @@ -16,6 +16,7 @@ package org.apache.spark.sql.rapids.execution +import ai.rapids.cudf.HostColumnVector.ColumnBuilder import ai.rapids.cudf.Table import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder @@ -25,8 +26,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.{MapPartitionsRDD, RDD} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.execution.SQLExecutionRDD -import org.apache.spark.sql.rapids.execution.GpuExternalRowToColumnConverter.{FixedWidthTypeConverter, VariableWidthTypeConverter} import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -39,15 +40,10 @@ private class GpuExternalRowToColumnConverter(schema: StructType) extends Serial f => GpuExternalRowToColumnConverter.getConverterForType(f.dataType, f.nullable) } - final def convert(row: Row, builders: GpuColumnarBatchBuilder): Long = { - var bytes: Long = 0 + final def convert(row: Row, builders: GpuColumnarBatchBuilder): Double = { + var bytes: Double = 0 for (idx <- 0 until row.length) { - converters(idx) match { - case tc: FixedWidthTypeConverter => - tc.append(row, idx, builders.builder(idx)) - case tc: VariableWidthTypeConverter => - bytes += tc.append(row, idx, builders.builder(idx)) - } + bytes += converters(idx).append(row, idx, builders.builder(idx)) } bytes } @@ -55,18 +51,31 @@ private class GpuExternalRowToColumnConverter(schema: StructType) extends Serial private object GpuExternalRowToColumnConverter { - private trait TypeConverter extends Serializable - - private abstract class FixedWidthTypeConverter extends TypeConverter { - /** Append row value to the column builder */ - def append(row: Row, column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit - } + // Sizes estimates for different things + /* + * size of an offset entry. In general we have 1 more offset entry than rows, so + * we might be off by one entry per column. + */ + private[this] val OFFSET = Integer.BYTES + private[this] val VALIDITY = 0.125 // 1/8th of a byte (1 bit) + private[this] val VALIDITY_N_OFFSET = OFFSET + VALIDITY - private abstract class VariableWidthTypeConverter extends TypeConverter { + private abstract class TypeConverter extends Serializable { /** Append row value to the column builder and return the number of data bytes written */ - def append(row: Row, column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Long + def append(row: Row, column: Int, builder: ColumnBuilder): Double + + /** + * This is here for structs. When you append a null to a struct the size is not known + * ahead of time. Also because structs push nulls down to the children this size should + * assume a validity even if the schema says it cannot be null. + */ + def getNullSize: Double } + private def getConverterFor(field: StructField): TypeConverter = + getConverterForType(field.dataType, field.nullable) + + private def getConverterForType(dataType: DataType, nullable: Boolean): TypeConverter = { (dataType, nullable) match { case (BooleanType, true) => BooleanConverter @@ -89,181 +98,329 @@ private object GpuExternalRowToColumnConverter { case (TimestampType, false) => NotNullLongConverter case (StringType, true) => StringConverter case (StringType, false) => NotNullStringConverter - // NOT SUPPORTED YET case CalendarIntervalType => CalendarConverter - // NOT SUPPORTED YET case at: ArrayType => new ArrayConverter( - // getConverterForType(at.elementType)) - // NOT SUPPORTED YET case st: StructType => new StructConverter(st.fields.map( - // (f) => getConverterForType(f.dataType))) - // NOT SUPPORTED YET case dt: DecimalType => new DecimalConverter(dt) - // NOT SUPPORTED YET case mt: MapType => new MapConverter(getConverterForType(mt.keyType), - // getConverterForType(mt.valueType)) + case (BinaryType, true) => BinaryConverter + case (BinaryType, false) => NotNullBinaryConverter + // NOT SUPPORTED YET + // case CalendarIntervalType => CalendarConverter + case (at: ArrayType, true) => + ArrayConverter(getConverterForType(at.elementType, at.containsNull)) + case (at: ArrayType, false) => + NotNullArrayConverter(getConverterForType(at.elementType, at.containsNull)) + case (st: StructType, true) => + StructConverter(st.fields.map(getConverterFor)) + case (st: StructType, false) => + NotNullStructConverter(st.fields.map(getConverterFor)) + case (dt: DecimalType, true) => + new DecimalConverter(dt.precision, dt.scale) + case (dt: DecimalType, false) => + new NotNullDecimalConverter(dt.precision, dt.scale) + case (MapType(k, v, vcn), true) => + MapConverter(getConverterForType(k, nullable = false), + getConverterForType(v, vcn)) + case (MapType(k, v, vcn), false) => + NotNullMapConverter(getConverterForType(k, nullable = false), + getConverterForType(v, vcn)) + case (NullType, true) => + NullConverter case (unknown, _) => throw new UnsupportedOperationException( s"Type $unknown not supported") } } + private object NullConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + builder.appendNull() + 1 + VALIDITY + } - private object BooleanConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + override def getNullSize: Double = 1 + VALIDITY + } + + private object BooleanConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullBooleanConverter.append(row, column, builder) } + 1 + VALIDITY + } + + override def getNullSize: Double = 1 + VALIDITY } - private object NotNullBooleanConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullBooleanConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(if (row.getBoolean(column)) 1.toByte else 0.toByte) + 1 + } + + override def getNullSize: Double = 1 + VALIDITY } - private object ByteConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object ByteConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullByteConverter.append(row, column, builder) } + 1 + VALIDITY + } + + override def getNullSize: Double = 1 + VALIDITY } - private object NotNullByteConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullByteConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getByte(column)) + 1 + } + + override def getNullSize: Double = 1 + VALIDITY } - private object ShortConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object ShortConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullShortConverter.append(row, column, builder) } + 2 + VALIDITY + } + + override def getNullSize: Double = 2 + VALIDITY } - private object NotNullShortConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullShortConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getShort(column)) + 2 + } + + override def getNullSize: Double = 2 + VALIDITY } - private object IntConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object IntConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullIntConverter.append(row, column, builder) } + 4 + VALIDITY + } + + override def getNullSize: Double = 4 + VALIDITY } - private object NotNullIntConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullIntConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getInt(column)) + 4 + } + + override def getNullSize: Double = 4 + VALIDITY } - private object FloatConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object FloatConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullFloatConverter.append(row, column, builder) } + 4 + VALIDITY + } + + override def getNullSize: Double = 4 + VALIDITY } - private object NotNullFloatConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullFloatConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getFloat(column)) + 4 + } + + override def getNullSize: Double = 4 + VALIDITY } - private object LongConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object LongConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullLongConverter.append(row, column, builder) } + 8 + VALIDITY + } + + override def getNullSize: Double = 8 + VALIDITY } - private object NotNullLongConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullLongConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getLong(column)) + 8 + } + + override def getNullSize: Double = 8 + VALIDITY } - private object DoubleConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object DoubleConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullDoubleConverter.append(row, column, builder) } + 8 + VALIDITY + } + + override def getNullSize: Double = 8 + VALIDITY } - private object NotNullDoubleConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullDoubleConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getDouble(column)) + 8 + } + + override def getNullSize: Double = 8 + VALIDITY } - private object StringConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object StringConverter extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = if (row.isNullAt(column)) { builder.appendNull() + VALIDITY_N_OFFSET } else { - NotNullStringConverter.append(row, column, builder) + NotNullStringConverter.append(row, column, builder) + VALIDITY } + + override def getNullSize: Double = VALIDITY_N_OFFSET } - private object NotNullStringConverter extends VariableWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Long = { + private object NotNullStringConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { val bytes = row.getString(column).getBytes builder.appendUTF8String(bytes) - bytes.length + bytes.length + OFFSET } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } +// + private object BinaryConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = + if (row.isNullAt(column)) { + builder.appendNull() + VALIDITY_N_OFFSET + } else { + NotNullBinaryConverter.append(row, column, builder) + VALIDITY + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + + private object NotNullBinaryConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + val child = builder.getChild(0) + val bytes = row.asInstanceOf[GenericRow].getSeq[Byte](column) + bytes.foreach(child.append) + builder.endList() + bytes.length + OFFSET + } + + override def getNullSize: Double = VALIDITY_N_OFFSET } + + private[this] def mapConvert( + keyConverter: TypeConverter, + valueConverter: TypeConverter, + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder) : Double = { + var ret = 0.0 + val m = row.getMap[Any, Any](column) + val numElements = m.size + val srcKeys = m.keys.toArray + val srcValues = m.values.toArray + val structBuilder = builder.getChild(0) + val keyBuilder = structBuilder.getChild(0) + val valueBuilder = structBuilder.getChild(1) + for (i <- 0 until numElements) { + ret += keyConverter.append(Row(srcKeys: _*), i, keyBuilder) + ret += valueConverter.append(Row(srcValues: _*), i, valueBuilder) + structBuilder.endStruct() + } + builder.endList() + ret + OFFSET + } + + private case class MapConverter( + keyConverter: TypeConverter, + valueConverter: TypeConverter) extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + if (row.isNullAt(column)) { + builder.appendNull() + VALIDITY_N_OFFSET + } else { + mapConvert(keyConverter, valueConverter, row, column, builder) + VALIDITY + } + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + + private case class NotNullMapConverter( + keyConverter: TypeConverter, + valueConverter: TypeConverter) extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = + mapConvert(keyConverter, valueConverter, row, column, builder) + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + // // private object CalendarConverter extends FixedWidthTypeConverter { // override def append( @@ -280,90 +437,125 @@ private object GpuExternalRowToColumnConverter { // } // } // } - // - // private case class ArrayConverter(childConverter: TypeConverter) extends TypeConverter { - // override def append( - // row: SpecializedGetters, - // column: Int, - // builder: ai.rapids.cudf.HostColumnVector.Builder): Unit = { - // if (row.isNullAt(column)) { - // builder.appendNull() - // } else { - // val values = row.getArray(column) - // val numElements = values.numElements() - // cv.appendArray(numElements) - // val arrData = cv.arrayData() - // for (i <- 0 until numElements) { - // childConverter.append(values, i, arrData) - // } - // } - // } - // } - // - // private case class StructConverter(childConverters: Array[TypeConverter]) - // extends TypeConverter { - // override def append(row: SpecializedGetters, - // column: Int, - // builder: ai.rapids.cudf.HostColumnVector.Builder): Unit = { - // if (row.isNullAt(column)) { - // builder.appendNull() - // } else { - // cv.appendStruct(false) - // val data = row.getStruct(column, childConverters.length) - // for (i <- 0 until childConverters.length) { - // childConverters(i).append(data, i, cv.getChild(i)) - // } - // } - // } - // } - // - // private case class DecimalConverter(dt: DecimalType) extends TypeConverter { - // override def append( - // row: SpecializedGetters, - // column: Int, - // builder: ai.rapids.cudf.HostColumnVector.Builder): Unit = { - // if (row.isNullAt(column)) { - // builder.appendNull() - // } else { - // val d = row.getDecimal(column, dt.precision, dt.scale) - // if (dt.precision <= Decimal.MAX_INT_DIGITS) { - // cv.appendInt(d.toUnscaledLong.toInt) - // } else if (dt.precision <= Decimal.MAX_LONG_DIGITS) { - // cv.appendLong(d.toUnscaledLong) - // } else { - // val integer = d.toJavaBigDecimal.unscaledValue - // val bytes = integer.toByteArray - // cv.appendByteArray(bytes, 0, bytes.length) - // } - // } - // } - // } - // - // private case class MapConverter(keyConverter: TypeConverter, valueConverter: TypeConverter) - // extends TypeConverter { - // override def append( - // row: SpecializedGetters, - // column: Int, - // builder: ai.rapids.cudf.HostColumnVector.Builder): Unit = { - // if (row.isNullAt(column)) { - // builder.appendNull() - // } else { - // val m = row.getMap(column) - // val keys = cv.getChild(0) - // val values = cv.getChild(1) - // val numElements = m.numElements() - // cv.appendArray(numElements) - // - // val srcKeys = m.keyArray() - // val srcValues = m.valueArray() - // - // for (i <- 0 until numElements) { - // keyConverter.append(srcKeys, i, keys) - // valueConverter.append(srcValues, i, values) - // } - // } - // } - // } + + private[this] def arrayConvert( + childConverter: TypeConverter, + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder) : Double = { + var ret = 0.0 + val values = row.getSeq(column) + val numElements = values.size + val child = builder.getChild(0) + for (i <- 0 until numElements) { + ret += childConverter.append(Row(values: _*), i, child) + } + builder.endList() + ret + OFFSET + } + + private case class ArrayConverter(childConverter: TypeConverter) + extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + if (row.isNullAt(column)) { + builder.appendNull() + VALIDITY_N_OFFSET + } else { + arrayConvert(childConverter, row, column, builder) + VALIDITY + } + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + + private case class NotNullArrayConverter(childConverter: TypeConverter) + extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + arrayConvert(childConverter, row, column, builder) + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + + private[this] def structConvert( + childConverters: Array[TypeConverter], + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder) : Double = { + var ret = 0.0 + val struct = row.getStruct(column) + for (i <- childConverters.indices) { + ret += childConverters(i).append(struct, i, builder.getChild(i)) + } + builder.endStruct() + ret + } + + private case class StructConverter( + childConverters: Array[TypeConverter]) extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + if (row.isNullAt(column)) { + builder.appendNull() + childConverters.map(_.getNullSize).sum + VALIDITY + // each child has to insert a null too, which is dependent on the child + } else { + structConvert(childConverters, row, column, builder) + VALIDITY + } + } + + override def getNullSize: Double = childConverters.map(_.getNullSize).sum + VALIDITY + } + + private case class NotNullStructConverter( + childConverters: Array[TypeConverter]) extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + structConvert(childConverters, row, column, builder) + } + + override def getNullSize: Double = childConverters.map(_.getNullSize).sum + VALIDITY + } + + private class DecimalConverter( + precision: Int, scale: Int) extends NotNullDecimalConverter(precision, scale) { + private val appendedSize = DecimalUtil.createCudfDecimal(precision, scale).getSizeInBytes + + VALIDITY + + override def append( + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + if (row.isNullAt(column)) { + builder.appendNull() + } else { + super.append(row, column, builder) + } + appendedSize + } + } + + private class NotNullDecimalConverter(precision: Int, scale: Int) extends TypeConverter { + private val appendedSize = DecimalUtil.createCudfDecimal(precision, scale).getSizeInBytes + + VALIDITY + + override def append( + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + val bigDecimal = row.getDecimal(column) + builder.append(bigDecimal) + appendedSize + } + + override def getNullSize: Double = { + appendedSize + VALIDITY + } + } } private class ExternalRowToColumnarIterator( @@ -372,11 +564,6 @@ private class ExternalRowToColumnarIterator( localGoal: CoalesceSizeGoal, converters: GpuExternalRowToColumnConverter) extends Iterator[ColumnarBatch] { - private val dataTypes: Array[DataType] = localSchema.fields.map(_.dataType) - private val variableWidthColumnCount = dataTypes.count(dt => !GpuBatchUtils.isFixedWidth(dt)) - private val fixedWidthDataSizePerRow = dataTypes.filter(GpuBatchUtils.isFixedWidth) - .map(_.defaultSize).sum - private val nullableColumns = localSchema.fields.count(_.nullable) private val targetSizeBytes = localGoal.targetSizeBytes private var targetRows = 0 private var totalOutputBytes: Long = 0 @@ -407,16 +594,11 @@ private class ExternalRowToColumnarIterator( val builders = new GpuColumnarBatchBuilder(localSchema, targetRows) try { var rowCount = 0 - var byteCount: Long = variableWidthColumnCount * 4 // offset bytes + // Double because validity can be < 1 byte, and this is just an estimate anyways + var byteCount: Double = 0 while (rowCount < targetRows && byteCount < targetSizeBytes && rowIter.hasNext) { val row = rowIter.next() - val variableWidthDataBytes = converters.convert(row, builders) - byteCount += fixedWidthDataSizePerRow // fixed-width data bytes - byteCount += variableWidthDataBytes // variable-width data bytes - byteCount += variableWidthColumnCount * GpuBatchUtils.OFFSET_BYTES // offset bytes - if (nullableColumns > 0 && rowCount % GpuBatchUtils.VALIDITY_BUFFER_BOUNDARY_ROWS == 0) { - byteCount += GpuBatchUtils.VALIDITY_BUFFER_BOUNDARY_BYTES * nullableColumns - } + byteCount += converters.convert(row, builders) rowCount += 1 } @@ -474,8 +656,10 @@ object InternalColumnarRddConverter extends Logging { def convert(df: DataFrame): RDD[Table] = { val schema = df.schema - if (!GpuOverrides.areAllSupportedTypes(schema.map(_.dataType) :_*)) { - val unsupported = schema.map(_.dataType).filter(!GpuOverrides.isSupportedType(_)).toSet + val unsupported = schema.map(_.dataType).filter( dt => !GpuOverrides.isSupportedType(dt, + allowMaps = true, allowStringMaps = true, allowNull = true, allowStruct = true, allowArray + = true, allowBinary = true, allowDecimal = true, allowNesting = true)).toSet + if (unsupported.nonEmpty) { throw new IllegalArgumentException(s"Cannot convert $df to GPU columnar $unsupported are " + s"not currently supported data types for columnar.") } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala index 3c8bfda7403..1a8a6579f68 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala @@ -22,9 +22,10 @@ import scala.util.Random import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder import org.scalatest.FunSuite +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.types.{DataTypes, Decimal, DecimalType, StructField, StructType} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, GenericRow} +import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String class GpuBatchUtilsSuite extends FunSuite { @@ -197,58 +198,139 @@ object GpuBatchUtilsSuite { rows.toArray } + def createExternalRows(schema: StructType, rowCount: Int): Array[Row] = { + val externalRows = new mutable.ArrayBuffer[Row](rowCount) + val r = new Random(0) + for (i <- 0 until rowCount) { + externalRows.append(new GenericRow(createExternalRowValues(i, r, schema.fields))) + } + externalRows.toArray + } + + private def createValueForType(i: Int, r: Random, dt: DataType, nullable: Boolean): Any = { + dt match { + case DataTypes.BooleanType => maybeNull(nullable, i, r.nextBoolean()) + case DataTypes.ByteType => maybeNull(nullable, i, r.nextInt().toByte) + case DataTypes.ShortType => maybeNull(nullable, i, r.nextInt().toShort) + case DataTypes.IntegerType => maybeNull(nullable, i, r.nextInt()) + case DataTypes.LongType => maybeNull(nullable, i, r.nextLong()) + case DataTypes.FloatType => maybeNull(nullable, i, r.nextFloat()) + case DataTypes.DoubleType => maybeNull(nullable, i, r.nextDouble()) + // Spark use Int to store a Date internally, so use nextInt to avoid + // 1). create Date object 2). convert Date to EpochDays int value + case DataTypes.DateType => maybeNull(nullable, i, r.nextInt()) + // Spark use Long to store a Timestamp internally, so use nextLong to avoid + // 1). create Timestamp object 2). convert Timestamp to microsecond long value + case DataTypes.TimestampType => maybeNull(nullable, i, r.nextLong()) + case dataType: DecimalType => + val upperBound = (0 until dataType.precision).foldLeft(1L)((x, _) => x * 10) + val unScaledValue = r.nextLong() % upperBound + maybeNull(nullable, i, Decimal(unScaledValue, dataType.precision, dataType.scale)) + case dataType@DataTypes.StringType => + if (nullable) { + // since we want a deterministic test that compares the estimate with actual + // usage we need to make sure the average length of strings is `dataType.defaultSize` + if (i % 2 == 0) { + null + } else { + createUTF8String(dataType.defaultSize * 2) + } + } else { + createUTF8String(dataType.defaultSize) + } + case dataType@DataTypes.BinaryType => + if (nullable) { + // since we want a deterministic test that compares the estimate with actual usage we + // need to make sure the average length of binary values is `dataType.defaultSize` + if (i % 2 == 0) { + null + } else { + r.nextString(dataType.defaultSize * 2).getBytes + } + } else { + r.nextString(dataType.defaultSize).getBytes + } + case ArrayType(elementType, containsNull) => + if (nullable && i % 2 == 0) { + null + } else { + val arrayValues = new mutable.ArrayBuffer[Any]() + for (_ <- 0 to r.nextInt(10)) { + arrayValues.append(createValueForType(i, r, elementType, containsNull)) + } + arrayValues.toArray.toSeq + } + case MapType(keyType, valueType, valueContainsNull) => + if (nullable && i % 2 == 0) { + null + } else { + // TODO: add other types + val map = mutable.Map[String, String]() + for ( j <- 0 until 10) { + if (valueContainsNull && j % 2 == 0) { + map += (createUTF8String(10).toString -> null) + } else { + map += (createUTF8String(10).toString -> createUTF8String(10).toString) + } + } + map + } + case StructType(fields) => + new GenericRow(fields.map(f => createValueForType(i, r, f.dataType, nullable))) + case unknown => throw new UnsupportedOperationException( + s"Type $unknown not supported") + } + } + + private def createRowValues(i: Int, r: Random, fields: Array[StructField]) = { + val values: Array[Any] = fields.map(field => { + createValueForType(i, r, field.dataType, field.nullable) + }) + values + } + + private def createExternalRowValues(i: Int, r: Random, fields: Array[StructField]): Array[Any] = { val values: Array[Any] = fields.map(field => { field.dataType match { - case DataTypes.BooleanType => maybeNull(field, i, r.nextBoolean()) - case DataTypes.ByteType => maybeNull(field, i, r.nextInt().toByte) - case DataTypes.ShortType => maybeNull(field, i, r.nextInt().toShort) - case DataTypes.IntegerType => maybeNull(field, i, r.nextInt()) - case DataTypes.LongType => maybeNull(field, i, r.nextLong()) - case DataTypes.FloatType => maybeNull(field, i, r.nextFloat()) - case DataTypes.DoubleType => maybeNull(field, i, r.nextDouble()) - case dataType: DecimalType => - val upperBound = (0 until dataType.precision).foldLeft(1L)((x, _) => x * 10) - val unScaledValue = r.nextLong() % upperBound - maybeNull(field, i, Decimal(unScaledValue, dataType.precision, dataType.scale)) - case dataType@DataTypes.StringType => - if (field.nullable) { - // since we want a deterministic test that compares the estimate with actual - // usage we need to make sure the average length of strings is `dataType.defaultSize` - if (i % 2 == 0) { - null - } else { - createString(dataType.defaultSize * 2) - } + // Since it's using the createUTF8String method for InternalRow case, need to convert to + // String for Row case. + case StringType => + val utf8StringOrNull = createValueForType(i, r, field.dataType, field.nullable) + if (utf8StringOrNull != null) { + utf8StringOrNull.asInstanceOf[UTF8String].toString } else { - createString(dataType.defaultSize) + utf8StringOrNull } - case dataType@DataTypes.BinaryType => - if (field.nullable) { - // since we want a deterministic test that compares the estimate with actual usage we - // need to make sure the average length of binary values is `dataType.defaultSize` - if (i % 2 == 0) { - null - } else { - r.nextString(dataType.defaultSize * 2).getBytes - } + case BinaryType => + val b = createValueForType(i, r, field.dataType, field.nullable) + if (b != null) { + b.asInstanceOf[Array[Byte]].toSeq + } else { + b + } + case DecimalType() => + val d = createValueForType(i, r, field.dataType, field.nullable) + if (d != null) { + d.asInstanceOf[Decimal].toJavaBigDecimal } else { - r.nextString(dataType.defaultSize).getBytes + d } + case _ => createValueForType(i, r, field.dataType, field.nullable) } }) values } - private def maybeNull(field: StructField, i: Int, value: Any): Any = { - if (field.nullable && i % 2 == 0) { + private def maybeNull(nullable: Boolean, i: Int, value: Any): Any = { + if (nullable && i % 2 == 0) { null } else { value } } - private def createString(size: Int): UTF8String = { + private def createUTF8String(size: Int): UTF8String = { // avoid multi byte characters to keep the test simple val str = (0 until size).map(_ => 'a').mkString UTF8String.fromString(str) diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala new file mode 100644 index 00000000000..7ce1d9fdc4f --- /dev/null +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala @@ -0,0 +1,276 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.execution + +import scala.collection.mutable + +import com.nvidia.spark.rapids.{ColumnarToRowIterator, GpuBatchUtilsSuite, NoopMetric, SparkQueryCompareTestSuite} +import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder + +import org.apache.spark.sql.catalyst.util.MapData +import org.apache.spark.sql.types._ + +class InternalColumnarRDDConverterSuite extends SparkQueryCompareTestSuite { + + def compareMapAndMapDate[K,V](map: collection.Map[K, V], mapData: MapData) = { + assert(map.size == mapData.numElements()) + val outputMap = mutable.Map[Any, Any]() + // Only String now, TODO: support other data types in Map + mapData.foreach(StringType, StringType, f = (k, v) => outputMap += (k.toString -> v.toString)) + val diff = outputMap.toSet diff map.toSet + assert(diff.toMap.isEmpty) + } + + test("transform binary data back and forth between Row and Columnar") { + val schema = StructType(Seq(StructField("Binary", BinaryType), + StructField("BinaryNotNull", BinaryType, nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getSeq[Byte](0) sameElements output.getBinary(0)) + } + assert(input.getSeq[Byte](1) sameElements output.getBinary(1)) + } + } + } + } + + test("transform boolean, byte, short, int, float, long, double, date, timestamp data" + + " back and forth between Row and Columnar") { + val schema = StructType(Seq( + StructField("Boolean", BooleanType), + StructField("BinaryNotNull", BooleanType, nullable = false), + StructField("Byte", ByteType), + StructField("ByteNotNull",ByteType, nullable = false), + StructField("Short", ShortType), + StructField("ShortNotNull", ShortType, nullable = false), + StructField("Int", IntegerType), + StructField("IntNotNull", IntegerType, nullable = false), + StructField("Float", FloatType), + StructField("FloatNotNull", FloatType, nullable = false), + StructField("Long", LongType), + StructField("LongNotNull", LongType, nullable = false), + StructField("Double", DoubleType), + StructField("DoubleNotNull", DoubleType, nullable = false), + StructField("Date", DateType), + StructField("DateNotNull", DateType, nullable = false), + StructField("Timestamp", TimestampType), + StructField("TimestampNotNull", TimestampType, nullable = false), + StructField("Decimal", DecimalType(20,10)), + StructField("DecimalNotNull", DecimalType(20,10), nullable = false) + )) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getBoolean(0) == output.getBoolean(0)) + } + assert(input.getBoolean(1) == output.getBoolean(1)) + + for ((f, i) <- schema.fields.zipWithIndex) { + if (f.nullable && input.isNullAt(i)) { + assert(output.isNullAt(i)) + } else { + if (f.dataType.isInstanceOf[DecimalType]) { + val l = input.get(i) + val r = output.get(i, f.dataType) + assert(input.get(i) == output.get(i, f.dataType) + .asInstanceOf[Decimal].toJavaBigDecimal) + } else { + assert(input.get(i) == output.get(i, f.dataType)) + } + } + } + } + } + } + } + + test("transform string data back and forth between Row and Columnar") { + val schema = StructType(Seq(StructField("String", StringType), + StructField("StringNotNull", StringType, nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getString(0) == output.getString(0)) + } + assert(input.getString(1) == output.getString(1)) + } + } + } + } + + + test("transform byte data back and forth between Row and Columnar") { + val schema = StructType(Seq(StructField("Byte", ByteType), + StructField("ByteNotNull", ByteType, nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getByte(0) == output.getByte(0)) + } + assert(input.getByte(1) == output.getByte(1)) + } + } + } + } + + test("transform array data back and forth between Row and Columnar") { + val schema = StructType(Seq(StructField("Array", ArrayType.apply(DoubleType)), + StructField("ArrayNotNull", ArrayType.apply(DoubleType, false), nullable = false))) + val numRows = 300 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getSeq(0) sameElements output.getArray(0).toDoubleArray()) + } + assert(input.getSeq(1) sameElements output.getArray(1).toDoubleArray()) + } + } + } + } + + test("transform map data back and forth between Row and Columnar") { + val schema = StructType(Seq( + StructField("Map", DataTypes.createMapType(StringType, StringType)), + StructField("MapNotNull", DataTypes.createMapType(StringType, StringType), + nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + compareMapAndMapDate(input.getMap(0), output.getMap(0)) + } + compareMapAndMapDate(input.getMap(1), output.getMap(1)) + } + } + } + } + + test("transform struct data back and forth between Row and Columnar") { + val structFieldArray = Array( + StructField("struct_int", IntegerType), + StructField("struct_double", DoubleType), + StructField("struct_array", DataTypes.createArrayType(DoubleType)) + ) + val schema = StructType(Seq( + StructField("Struct", DataTypes.createStructType(structFieldArray)), + StructField("StructNotNull", DataTypes.createStructType(structFieldArray), + nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + val inputStructRow = input.getStruct(0) + val outputStructRow = output.getStruct(0, 3) + if (inputStructRow.isNullAt(0)) { + assert(outputStructRow.isNullAt(0)) + } else { + assert(inputStructRow.getInt(0) == outputStructRow.getInt(0)) + } + if (inputStructRow.isNullAt(1)) { + assert(outputStructRow.isNullAt(1)) + } else { + assert(inputStructRow.getDouble(1) == outputStructRow.getDouble(1)) + } + if (inputStructRow.isNullAt(2)) { + assert(outputStructRow.isNullAt(2)) + } else { + assert(inputStructRow.getSeq(2) sameElements outputStructRow.getArray(2) + .toDoubleArray()) + } + } + } + } + } + } + +} + From 1e95655d7fee525bfb12dd445118ba18f5b18296 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 7 Dec 2021 17:18:32 +0800 Subject: [PATCH 33/52] [BUG] Update nightly test script to dynamically set mem_fraction [skip ci] (#4317) * Decrease parallelism and update nightly script to dynamically set mem fraction Signed-off-by: Peixin Li * reorder the cases --- jenkins/spark-tests.sh | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index 9f6379e87c3..d110f83131b 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -139,14 +139,16 @@ export SEQ_CONF="--executor-memory 16G \ # currently we hardcode the parallelism and configs based on our CI node's hardware specs, # we can make it dynamically generated if this script is going to be used in other scenarios in the future +PARALLELISM=${PARALLELISM:-'4'} +MEMORY_FRACTION=$(python -c "print(1/($PARALLELISM + 0.2))") export PARALLEL_CONF="--executor-memory 4G \ --total-executor-cores 2 \ --conf spark.executor.cores=2 \ --conf spark.task.cpus=1 \ --conf spark.rapids.sql.concurrentGpuTasks=2 \ ---conf spark.rapids.memory.gpu.allocFraction=0.15 \ --conf spark.rapids.memory.gpu.minAllocFraction=0 \ ---conf spark.rapids.memory.gpu.maxAllocFraction=0.15" +--conf spark.rapids.memory.gpu.allocFraction=${MEMORY_FRACTION} \ +--conf spark.rapids.memory.gpu.maxAllocFraction=${MEMORY_FRACTION}" export CUDF_UDF_TEST_ARGS="--conf spark.rapids.memory.gpu.allocFraction=0.1 \ --conf spark.rapids.memory.gpu.minAllocFraction=0 \ @@ -213,13 +215,13 @@ if [[ $TEST_MODE == "ALL" || $TEST_MODE == "IT_ONLY" ]]; then # integration tests if [[ $PARALLEL_TEST == "true" ]] && [ -x "$(command -v parallel)" ]; then # put most time-consuming tests at the head of queue - time_consuming_tests="join_test.py generate_expr_test.py parquet_write_test.py" + time_consuming_tests="join_test.py hash_aggregate_test.py parquet_write_test.py" tests_list=$(find "$SCRIPT_PATH"/src/main/python/ -name "*_test.py" -printf "%f ") tests=$(echo "$time_consuming_tests $tests_list" | tr ' ' '\n' | awk '!x[$0]++' | xargs) # --halt "now,fail=1": exit when the first job fail, and kill running jobs. # we can set it to "never" and print failed ones after finish running all tests if needed # --group: print stderr after test finished for better readability - parallel --group --halt "now,fail=1" -j5 run_test ::: $tests + parallel --group --halt "now,fail=1" -j"${PARALLELISM}" run_test ::: $tests else run_test all fi From a59876039f9c5709c6f6ddb8482558fe9918333b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 7 Dec 2021 15:04:31 -0700 Subject: [PATCH 34/52] Optimize AQE plan to avoid redundant transitions (#3144) Signed-off-by: Andy Grove --- .../rapids/shims/v2/Spark30XdbShims.scala | 11 ++ .../spark/rapids/shims/v2/Spark30XShims.scala | 10 ++ .../v2/AvoidAdaptiveTransitionToRow.scala | 84 +++++++++ .../spark/rapids/shims/v2/Spark31XShims.scala | 10 ++ .../rapids/shims/v2/Spark31XdbShims.scala | 11 ++ .../spark/rapids/shims/v2/Spark32XShims.scala | 7 + .../spark/rapids/GpuTransitionOverrides.scala | 163 ++++++------------ .../com/nvidia/spark/rapids/SparkShims.scala | 9 + .../spark/rapids/AdaptiveQueryExecSuite.scala | 20 ++- 9 files changed, 210 insertions(+), 115 deletions(-) create mode 100644 sql-plugin/src/main/301until320-all/scala/com/nvidia/spark/rapids/shims/v2/AvoidAdaptiveTransitionToRow.scala diff --git a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala index 66b32106c1c..39cbffeee55 100644 --- a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala +++ b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala @@ -707,4 +707,15 @@ abstract class Spark30XdbShims extends Spark30XdbShimsBase with Logging { } override def getLegacyStatisticalAggregate(): Boolean = true + + override def supportsColumnarAdaptivePlans: Boolean = false + + override def columnarAdaptivePlan(a: AdaptiveSparkPlanExec, goal: CoalesceSizeGoal): SparkPlan = { + // When the input is an adaptive plan we do not get to see the GPU version until + // the plan is executed and sometimes the plan will have a GpuColumnarToRowExec as the + // final operator and we can bypass this to keep the data columnar by inserting + // the [[AvoidAdaptiveTransitionToRow]] operator here + AvoidAdaptiveTransitionToRow(GpuRowToColumnarExec(a, goal)) + } + } diff --git a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala index 18680d3b902..78b50d670b5 100644 --- a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala +++ b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala @@ -360,4 +360,14 @@ abstract class Spark30XShims extends Spark301util320Shims with Logging { override def getAdaptiveInputPlan(adaptivePlan: AdaptiveSparkPlanExec): SparkPlan = { adaptivePlan.initialPlan } + + override def supportsColumnarAdaptivePlans: Boolean = false + + override def columnarAdaptivePlan(a: AdaptiveSparkPlanExec, goal: CoalesceSizeGoal): SparkPlan = { + // When the input is an adaptive plan we do not get to see the GPU version until + // the plan is executed and sometimes the plan will have a GpuColumnarToRowExec as the + // final operator and we can bypass this to keep the data columnar by inserting + // the [[AvoidAdaptiveTransitionToRow]] operator here + AvoidAdaptiveTransitionToRow(GpuRowToColumnarExec(a, goal)) + } } diff --git a/sql-plugin/src/main/301until320-all/scala/com/nvidia/spark/rapids/shims/v2/AvoidAdaptiveTransitionToRow.scala b/sql-plugin/src/main/301until320-all/scala/com/nvidia/spark/rapids/shims/v2/AvoidAdaptiveTransitionToRow.scala new file mode 100644 index 00000000000..626f027ad4a --- /dev/null +++ b/sql-plugin/src/main/301until320-all/scala/com/nvidia/spark/rapids/shims/v2/AvoidAdaptiveTransitionToRow.scala @@ -0,0 +1,84 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.nvidia.spark.rapids.shims.v2 + +import java.lang.reflect.Method + +import com.nvidia.spark.rapids.{GpuColumnarToRowExec, GpuExec, GpuRowToColumnarExec} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** + * This operator will attempt to optimize the case when we are writing the results of + * an adaptive query to disk so that we remove the redundant transitions from columnar + * to row within AdaptiveSparkPlanExec followed by a row to columnar transition. + * + * Specifically, this is the plan we see in this case: + * + * {{{ + * GpuRowToColumnar(AdaptiveSparkPlanExec(GpuColumnarToRow(child)) + * }}} + * + * We perform this optimization at runtime rather than during planning, because when the adaptive + * plan is being planned and executed, we don't know whether it is being called from an operation + * that wants rows (such as CollectTailExec) or from an operation that wants columns (such as + * GpuDataWritingCommandExec). + * + * Spark does not provide a mechanism for executing an adaptive plan and retrieving columnar + * results and the internal methods that we need to call are private, so we use reflection to + * call them. + * + * @param child The plan to execute + */ +case class AvoidAdaptiveTransitionToRow(child: SparkPlan) extends ShimUnaryExecNode with GpuExec { + + override def doExecute(): RDD[InternalRow] = + throw new IllegalStateException(s"Row-based execution should not occur for $this") + + override def output: Seq[Attribute] = child.output + + override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child match { + case GpuRowToColumnarExec(a: AdaptiveSparkPlanExec, _, _) => + val getFinalPhysicalPlan = getPrivateMethod("getFinalPhysicalPlan") + val plan = getFinalPhysicalPlan.invoke(a) + val rdd = plan match { + case t: GpuColumnarToRowExec => + t.child.executeColumnar() + case _ => + child.executeColumnar() + } + + // final UI update + val finalPlanUpdate = getPrivateMethod("finalPlanUpdate") + finalPlanUpdate.invoke(a) + + rdd + + case _ => + child.executeColumnar() + } + + private def getPrivateMethod(name: String): Method = { + val m = classOf[AdaptiveSparkPlanExec].getDeclaredMethod(name) + m.setAccessible(true) + m + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala index 015730379ab..0d2ec7c80ee 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala @@ -519,4 +519,14 @@ abstract class Spark31XShims extends Spark301util320Shims with Logging { SQLConf.get.legacyStatisticalAggregate override def hasCastFloatTimestampUpcast: Boolean = false + + override def supportsColumnarAdaptivePlans: Boolean = false + + override def columnarAdaptivePlan(a: AdaptiveSparkPlanExec, goal: CoalesceSizeGoal): SparkPlan = { + // When the input is an adaptive plan we do not get to see the GPU version until + // the plan is executed and sometimes the plan will have a GpuColumnarToRowExec as the + // final operator and we can bypass this to keep the data columnar by inserting + // the [[AvoidAdaptiveTransitionToRow]] operator here + AvoidAdaptiveTransitionToRow(GpuRowToColumnarExec(a, goal)) + } } diff --git a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala index e8b4e57457a..6649c12b498 100644 --- a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala +++ b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala @@ -849,4 +849,15 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { override def getLegacyStatisticalAggregate(): Boolean = SQLConf.get.legacyStatisticalAggregate + + override def supportsColumnarAdaptivePlans: Boolean = false + + override def columnarAdaptivePlan(a: AdaptiveSparkPlanExec, goal: CoalesceSizeGoal): SparkPlan = { + // When the input is an adaptive plan we do not get to see the GPU version until + // the plan is executed and sometimes the plan will have a GpuColumnarToRowExec as the + // final operator and we can bypass this to keep the data columnar by inserting + // the [[AvoidAdaptiveTransitionToRow]] operator here + AvoidAdaptiveTransitionToRow(GpuRowToColumnarExec(a, goal)) + } + } diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala index 231dde4e6f4..452618d74b8 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala @@ -1086,4 +1086,11 @@ trait Spark32XShims extends SparkShims with Logging { override def getAdaptiveInputPlan(adaptivePlan: AdaptiveSparkPlanExec): SparkPlan = { adaptivePlan.initialPlan } + + override def columnarAdaptivePlan(a: AdaptiveSparkPlanExec, + goal: CoalesceSizeGoal): SparkPlan = { + a.copy(supportsColumnar = true) + } + + override def supportsColumnarAdaptivePlans: Boolean = true } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala index 09041d8a9ef..566707b0dd9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala @@ -16,14 +16,8 @@ package com.nvidia.spark.rapids -import java.lang.reflect.Method - import scala.annotation.tailrec -import com.nvidia.spark.rapids.shims.v2.ShimUnaryExecNode - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeReference, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, SortOrder} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ @@ -34,8 +28,7 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, Exchange, ReusedExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} import org.apache.spark.sql.rapids.{GpuDataSourceScanExec, GpuFileSourceScanExec, GpuInputFileBlockLength, GpuInputFileBlockStart, GpuInputFileName, GpuShuffleEnv} -import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastToCpuExec, GpuCustomShuffleReaderExec, GpuHashJoin, GpuShuffleExchangeExecBase} -import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastExchangeExecBase, GpuBroadcastToCpuExec, GpuCustomShuffleReaderExec, GpuHashJoin, GpuShuffleExchangeExecBase} /** * Rules that run after the row to columnar and columnar to row transitions have been inserted. @@ -82,48 +75,64 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { def optimizeAdaptiveTransitions( plan: SparkPlan, parent: Option[SparkPlan]): SparkPlan = plan match { + + case GpuBringBackToHost(child) if parent.isEmpty => + // This is hacky but we need to remove the GpuBringBackToHost from the final + // query stage, if there is one. It gets inserted by + // GpuTransitionOverrides.insertColumnarFromGpu around columnar adaptive + // plans when we are writing to columnar formats on the GPU. It would be nice to avoid + // inserting it in the first place but we just don't have enough context + // at the time GpuTransitionOverrides is applying rules. + child + // HostColumnarToGpu(RowToColumnarExec(..)) => GpuRowToColumnarExec(..) case HostColumnarToGpu(r2c: RowToColumnarExec, goal) => val child = optimizeAdaptiveTransitions(r2c.child, Some(r2c)) - val preProcessing = child.getTagValue(GpuOverrides.preRowToColProjection) - .getOrElse(Seq.empty) - val transition = GpuRowToColumnarExec(child, goal, preProcessing) - r2c.child match { - case _: AdaptiveSparkPlanExec => - // When the input is an adaptive plan we do not get to see the GPU version until - // the plan is executed and sometimes the plan will have a GpuColumnarToRowExec as the - // final operator and we can bypass this to keep the data columnar by inserting - // the [[AvoidAdaptiveTransitionToRow]] operator here - AvoidAdaptiveTransitionToRow(transition) + child match { + case a: AdaptiveSparkPlanExec => + // we hit this case when we have an adaptive plan wrapped in a write + // to columnar file format on the GPU + val columnarAdaptivePlan = ShimLoader.getSparkShims.columnarAdaptivePlan(a, goal) + optimizeAdaptiveTransitions(columnarAdaptivePlan, None) case _ => - transition + val preProcessing = child.getTagValue(GpuOverrides.preRowToColProjection) + .getOrElse(Seq.empty) + GpuRowToColumnarExec(child, goal, preProcessing) } - case ColumnarToRowExec(GpuBringBackToHost( - GpuShuffleCoalesceExec(e: GpuShuffleExchangeExecBase, _))) if parent.isEmpty => - // We typically want the final operator in the plan (the operator that has no parent) to be - // wrapped in `ColumnarToRowExec(GpuBringBackToHost(ShuffleCoalesceExec(_)))` operators to - // bring the data back onto the host and be translated to rows so that it can be returned - // from the Spark API. However, in the case of AQE, each exchange operator is treated as an - // individual query with no parent and we need to remove these operators in this case - // because we need to return an operator that implements `BroadcastExchangeLike` or - // `ShuffleExchangeLike`. The coalesce step gets added back into the plan later on, in a - // future query stage that reads the output from this query stage. This is handled in the - // case clauses below. - e.withNewChildren(e.children.map(c => optimizeAdaptiveTransitions(c, Some(e)))) - - case ColumnarToRowExec(GpuBringBackToHost( - GpuCoalesceBatches(e: GpuShuffleExchangeExecBase, _))) if parent.isEmpty => - // We typically want the final operator in the plan (the operator that has no parent) to be - // wrapped in `ColumnarToRowExec(GpuBringBackToHost(GpuCoalesceBatches(_)))` operators to - // bring the data back onto the host and be translated to rows so that it can be returned - // from the Spark API. However, in the case of AQE, each exchange operator is treated as an - // individual query with no parent and we need to remove these operators in this case - // because we need to return an operator that implements `BroadcastExchangeLike` or - // `ShuffleExchangeLike`. The coalesce step gets added back into the plan later on, in a - // future query stage that reads the output from this query stage. This is handled in the - // case clauses below. - e.withNewChildren(e.children.map(c => optimizeAdaptiveTransitions(c, Some(e)))) + // adaptive plan final query stage with columnar output + case r2c @ RowToColumnarExec(child) if parent.isEmpty => + val optimizedChild = optimizeAdaptiveTransitions(child, Some(r2c)) + val preProcessing = optimizedChild.getTagValue(GpuOverrides.preRowToColProjection) + .getOrElse(Seq.empty) + GpuRowToColumnarExec(optimizedChild, TargetSize(rapidsConf.gpuTargetBatchSizeBytes), + preProcessing) + + case ColumnarToRowExec(bb: GpuBringBackToHost) => + // We typically want the final operator in the plan (the operator that has no parent) to be + // wrapped in `ColumnarToRowExec(GpuBringBackToHost(_))` operators to + // bring the data back onto the host and be translated to rows so that it can be returned + // from the Spark API. However, in the case of AQE, each exchange operator is treated as an + // individual query with no parent and we need to remove these operators in this case + // because we need to return an operator that implements `BroadcastExchangeLike` or + // `ShuffleExchangeLike`. + bb.child match { + case GpuShuffleCoalesceExec(e: GpuShuffleExchangeExecBase, _) if parent.isEmpty => + // The coalesce step gets added back into the plan later on, in a + // future query stage that reads the output from this query stage. This + // is handled in the case clauses below. + e.withNewChildren(e.children.map(c => optimizeAdaptiveTransitions(c, Some(e)))) + case GpuCoalesceBatches(e: GpuShuffleExchangeExecBase, _) if parent.isEmpty => + // The coalesce step gets added back into the plan later on, in a + // future query stage that reads the output from this query stage. This + // is handled in the case clauses below. + e.withNewChildren(e.children.map(c => optimizeAdaptiveTransitions(c, Some(e)))) + case _ => optimizeAdaptiveTransitions(bb.child, Some(bb)) match { + case e: GpuBroadcastExchangeExecBase => e + case e: GpuShuffleExchangeExecBase => e + case other => getColumnarToRowExec(other) + } + } case s: ShuffleQueryStageExec => // When reading a materialized shuffle query stage in AQE mode, we need to insert an @@ -170,13 +179,6 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { case ColumnarToRowExec(e: ShuffleQueryStageExec) => getColumnarToRowExec(optimizeAdaptiveTransitions(e, Some(plan))) - case ColumnarToRowExec(bb: GpuBringBackToHost) => - optimizeAdaptiveTransitions(bb.child, Some(bb)) match { - case e: GpuBroadcastExchangeExec => e - case e: GpuShuffleExchangeExecBase => e - case other => getColumnarToRowExec(other) - } - // inserts postColumnarToRowTransition into newly-created GpuColumnarToRowExec case p if p.getTagValue(GpuOverrides.postColToRowProjection).nonEmpty => val c2r = p.children.map(optimizeAdaptiveTransitions(_, Some(p))).head @@ -613,61 +615,4 @@ object GpuTransitionOverrides { case _: InputFileBlockLength => true case e => e.children.exists(checkHasInputFileExpressions) } -} - -/** - * This operator will attempt to optimize the case when we are writing the results of - * an adaptive query to disk so that we remove the redundant transitions from columnar - * to row within AdaptiveSparkPlanExec followed by a row to columnar transition. - * - * Specifically, this is the plan we see in this case: - * - * {{{ - * GpuRowToColumnar(AdaptiveSparkPlanExec(GpuColumnarToRow(child)) - * }}} - * - * We perform this optimization at runtime rather than during planning, because when the adaptive - * plan is being planned and executed, we don't know whether it is being called from an operation - * that wants rows (such as CollectTailExec) or from an operation that wants columns (such as - * GpuDataWritingCommandExec). - * - * Spark does not provide a mechanism for executing an adaptive plan and retrieving columnar - * results and the internal methods that we need to call are private, so we use reflection to - * call them. - * - * @param child The plan to execute - */ -case class AvoidAdaptiveTransitionToRow(child: SparkPlan) extends ShimUnaryExecNode with GpuExec { - - override def doExecute(): RDD[InternalRow] = - throw new IllegalStateException(s"Row-based execution should not occur for $this") - - override def output: Seq[Attribute] = child.output - - override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child match { - case GpuRowToColumnarExec(a: AdaptiveSparkPlanExec, _, _) => - val getFinalPhysicalPlan = getPrivateMethod("getFinalPhysicalPlan") - val plan = getFinalPhysicalPlan.invoke(a) - val rdd = plan match { - case t: GpuColumnarToRowExec => - t.child.executeColumnar() - case _ => - child.executeColumnar() - } - - // final UI update - val finalPlanUpdate = getPrivateMethod("finalPlanUpdate") - finalPlanUpdate.invoke(a) - - rdd - - case _ => - child.executeColumnar() - } - - private def getPrivateMethod(name: String): Method = { - val m = classOf[AdaptiveSparkPlanExec].getDeclaredMethod(name) - m.setAccessible(true) - m - } -} +} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index f7d851c4451..d18bac8cf61 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -298,4 +298,13 @@ trait SparkShims { def timestampFormatInRead(csvOpts: CSVOptions): Option[String] def neverReplaceShowCurrentNamespaceCommand: ExecRule[_ <: SparkPlan] + + /** + * Determine if the Spark version allows the supportsColumnar flag to be overridden + * in AdaptiveSparkPlanExec. This feature was introduced in Spark 3.2 as part of + * SPARK-35881. + */ + def supportsColumnarAdaptivePlans: Boolean + + def columnarAdaptivePlan(a: AdaptiveSparkPlanExec, goal: CoalesceSizeGoal): SparkPlan } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala index b920b201a5d..bb494f9e9d1 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, Exchange, import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions.{col, when} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.rapids.execution.GpuCustomShuffleReaderExec import org.apache.spark.sql.types.{ArrayType, DataTypes, DecimalType, IntegerType, StringType, StructField, StructType} @@ -329,14 +330,21 @@ class AdaptiveQueryExecSuite _.isInstanceOf[AdaptiveSparkPlanExec]) .get.asInstanceOf[AdaptiveSparkPlanExec] - val transition = adaptiveSparkPlanExec + if (ShimLoader.getSparkShims.supportsColumnarAdaptivePlans) { + // we avoid the transition entirely with Spark 3.2+ due to the changes in SPARK-35881 to + // support columnar adaptive plans + assert(adaptiveSparkPlanExec .executedPlan - .asInstanceOf[GpuColumnarToRowExec] - - // although the plan contains a GpuColumnarToRowExec, we bypass it in - // AvoidAdaptiveTransitionToRow so the metrics should reflect that - assert(transition.metrics("numOutputRows").value === 0) + .isInstanceOf[GpuFileSourceScanExec]) + } else { + val transition = adaptiveSparkPlanExec + .executedPlan + .asInstanceOf[GpuColumnarToRowExec] + // although the plan contains a GpuColumnarToRowExec, we bypass it in + // AvoidAdaptiveTransitionToRow so the metrics should reflect that + assert(transition.metrics("numOutputRows").value === 0) + } }, conf) } From 3bc61dc044c86fe242762734ebea91b8b5bb1854 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Wed, 8 Dec 2021 08:14:49 +0800 Subject: [PATCH 35/52] Use data type to create an inspector for a foldable GPU expression. (#4282) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Use data type to create an inspector for a foldable GPU expression。 Signed-off-by: Firestarman --- .../src/main/python/row-based_udf_test.py | 2 +- .../sql/hive/rapids/rowBasedHiveUDFs.scala | 45 ++++++++++++++++--- .../rapids/udf/hive/EmptyHiveSimpleUDF.java | 4 +- 3 files changed, 43 insertions(+), 8 deletions(-) diff --git a/integration_tests/src/main/python/row-based_udf_test.py b/integration_tests/src/main/python/row-based_udf_test.py index 45ae6704746..5cf2cdfdd58 100644 --- a/integration_tests/src/main/python/row-based_udf_test.py +++ b/integration_tests/src/main/python/row-based_udf_test.py @@ -28,7 +28,7 @@ def evalfn(spark): assert_gpu_and_cpu_are_equal_sql( evalfn, "hive_simple_udf_test_table", - "SELECT i, emptysimple(s) FROM hive_simple_udf_test_table", + "SELECT i, emptysimple(s, 'const_string') FROM hive_simple_udf_test_table", conf={'spark.rapids.sql.rowBasedUDF.enabled': 'true'}) def test_hive_empty_generic_udf(): diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala index cc0ddfcbd86..85cb9c09c9b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala @@ -18,17 +18,17 @@ package org.apache.spark.sql.hive.rapids import scala.collection.JavaConverters._ -import com.nvidia.spark.rapids.GpuRowBasedUserDefinedFunction +import com.nvidia.spark.rapids.{GpuExpression, GpuLiteral, GpuRowBasedUserDefinedFunction, GpuScalar} import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, UDF} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic.GenericUDF import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper -import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, ObjectInspector, ObjectInspectorFactory} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Expression, SpecializedGetters} +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, SpecializedGetters} import org.apache.spark.sql.hive.{DeferredObjectAdapter, HiveInspectors} import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.types.DataType @@ -55,6 +55,41 @@ trait GpuRowBasedHiveUDFBase extends GpuRowBasedUserDefinedFunction with HiveIns override def prettyName: String = name + /** Create an object inspector from a GPU expression. */ + protected def gpuToInspector(expr: Expression): ObjectInspector = expr match { + case GpuLiteral(value, dataType) => + // Convert GpuLiterals to CPU Literals as possible as we can for primitive types, to + // leverage the Spark `toInspector(Expression)` method as much as possible. + // Because the `toInspector(Expression)` method will take care of the CPU Literal + // especially, converting it to a ConstantObjectInspector when it is primitive type. A + // `ConstantObjectInspector` can accelerate the row data reading by caching the actual + // value and skipping the null check which becomes unnecessary. + value match { + case scalar: ai.rapids.cudf.Scalar => + if (scalar.getType.isNestedType) { + // Nested type, so create an inspector from the data type. + toInspector(dataType) + } else { + try { + toInspector(Literal.create(GpuScalar.extract(scalar), dataType)) + } catch { + // Unsupported type for extraction, so use the data type way instead. + case _: UnsupportedOperationException => toInspector(dataType) + } + } + case _ => toInspector(Literal.create(value, dataType)) + } + case ge: GpuExpression if ge.foldable => + // Create an inspector from the data type instead, to avoid evaluation on the driver side, + // which will be triggered inside the `toInspector(Expression)` method for a foldable + // expression. Because GPU expressions should not be evaluated on the driver side. + toInspector(ge.dataType) + case _ => + // For other expressions, it is safe to call `toInspector(Expression)`, which will call into + // `toInspector(DataType)` directly for now. + toInspector(expr) + } + @transient protected lazy val childRowAccessors: Array[SpecializedGetters => Any] = children.zipWithIndex.map { case (child, i) => @@ -63,7 +98,7 @@ trait GpuRowBasedHiveUDFBase extends GpuRowBasedUserDefinedFunction with HiveIns }.toArray @transient - protected lazy val argumentInspectors = children.map(toInspector) + protected lazy val argumentInspectors = children.map(gpuToInspector) } /** Row-based version of Spark's `HiveSimpleUDF` running in a GPU operation */ @@ -77,7 +112,7 @@ case class GpuRowBasedHiveSimpleUDF( override lazy val function: UDF = funcWrapper.createFunction[UDF]() @transient - private lazy val wrappers = children.map(x => wrapperFor(toInspector(x), x.dataType)).toArray + private lazy val wrappers = children.map(x => wrapperFor(gpuToInspector(x), x.dataType)).toArray @transient private lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) diff --git a/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java index 43c494e2ecf..f2c5a51a49f 100644 --- a/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java +++ b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java @@ -18,9 +18,9 @@ import org.apache.hadoop.hive.ql.exec.UDF; -/** An empty Hive simple UDF returning the input directly for row-based UDF test only. */ +/** An empty Hive simple UDF returning the first input directly for row-based UDF test only. */ public class EmptyHiveSimpleUDF extends UDF { - public String evaluate(String in) { + public String evaluate(String in, String in2) { return in; } } From 367c5d7f7705a9b5e9eaf6ecc792b29f9ef39e6b Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Wed, 8 Dec 2021 09:36:50 +0800 Subject: [PATCH 36/52] Check revisions for all shim jars while build all (#4266) * Check revisions for all shim jars while build all Signed-off-by: Chong Gao --- dist/maven-antrun/build-parallel-worlds.xml | 9 +++- dist/scripts/check-shims-revisions.sh | 52 +++++++++++++++++++++ 2 files changed, 60 insertions(+), 1 deletion(-) create mode 100755 dist/scripts/check-shims-revisions.sh diff --git a/dist/maven-antrun/build-parallel-worlds.xml b/dist/maven-antrun/build-parallel-worlds.xml index 66fedb8671f..32b77f69480 100644 --- a/dist/maven-antrun/build-parallel-worlds.xml +++ b/dist/maven-antrun/build-parallel-worlds.xml @@ -121,6 +121,13 @@ + + + + + + @@ -149,4 +156,4 @@ - \ No newline at end of file + diff --git a/dist/scripts/check-shims-revisions.sh b/dist/scripts/check-shims-revisions.sh new file mode 100755 index 00000000000..b4b0aef3359 --- /dev/null +++ b/dist/scripts/check-shims-revisions.sh @@ -0,0 +1,52 @@ +#!/bin/bash + +# +# Copyright (c) 2021, NVIDIA CORPORATION. All rights reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +set -e +# +# check the revisions of each shims, they should be equal +# $1 is included_buildvers +function check-shims-revisions() { + included_buildvers="$1" + # PWD should be spark-rapids root path + parallel_dir=${PWD}/parallel-world + pre_revision="" + pre_shim_version_path="" + + IFS="," + for shim in ${included_buildvers}; do + # trim + shim=$(echo "${shim}" | xargs) + shim_version_path="${parallel_dir}/spark${shim}/rapids4spark-version-info.properties" + if [[ -f "$shim_version_path" ]] ; then + curr_revision=$(grep "revision=" "${shim_version_path}" | cut -d'=' -f2) + if [ -n "$pre_revision" ] && [[ "$curr_revision" != "$pre_revision" ]] ; then + echo + echo "Check Failed: git revisions between shims are not equal" + echo "Please check the revisions of each shim to see which one is inconsistent. Note, if building with Databricks those jars are built separately." + exit 1 + fi + pre_revision="${curr_revision}" + pre_shim_version_path="${shim_version_path}" + else + echo "Error: version file missing: ${shim_version_path}" + exit 1 + fi + done +} + +check-shims-revisions "$1" From 51a083c09b04ccaa088275a9651d3fea6d034413 Mon Sep 17 00:00:00 2001 From: Peixin Date: Wed, 8 Dec 2021 14:28:22 +0800 Subject: [PATCH 37/52] Fix 4315 decrease concurrentGpuTasks to avoid sum test OOM (#4326) * Nightly test concurrentGpuTasks to 1 for OOM Signed-off-by: Peixin Li * update warning filter for pytest-order * reduce executor core num * reorder tests --- integration_tests/pytest.ini | 2 ++ jenkins/spark-tests.sh | 8 ++++---- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/integration_tests/pytest.ini b/integration_tests/pytest.ini index b2797db5ee0..19814c1796b 100644 --- a/integration_tests/pytest.ini +++ b/integration_tests/pytest.ini @@ -26,3 +26,5 @@ markers = validate_execs_in_gpu_plan([execs]): Exec class names to validate they exist in the GPU plan. shuffle_test: Mark to include test in the RAPIDS Shuffle Manager premerge_ci_1: Mark test that will run in first k8s pod in case of parallel build premerge job +filterwarnings = + ignore:.*pytest.mark.order.*:_pytest.warning_types.PytestUnknownMarkWarning diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index d110f83131b..1387be78d66 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -142,10 +142,10 @@ export SEQ_CONF="--executor-memory 16G \ PARALLELISM=${PARALLELISM:-'4'} MEMORY_FRACTION=$(python -c "print(1/($PARALLELISM + 0.2))") export PARALLEL_CONF="--executor-memory 4G \ ---total-executor-cores 2 \ ---conf spark.executor.cores=2 \ +--total-executor-cores 1 \ +--conf spark.executor.cores=1 \ --conf spark.task.cpus=1 \ ---conf spark.rapids.sql.concurrentGpuTasks=2 \ +--conf spark.rapids.sql.concurrentGpuTasks=1 \ --conf spark.rapids.memory.gpu.minAllocFraction=0 \ --conf spark.rapids.memory.gpu.allocFraction=${MEMORY_FRACTION} \ --conf spark.rapids.memory.gpu.maxAllocFraction=${MEMORY_FRACTION}" @@ -215,7 +215,7 @@ if [[ $TEST_MODE == "ALL" || $TEST_MODE == "IT_ONLY" ]]; then # integration tests if [[ $PARALLEL_TEST == "true" ]] && [ -x "$(command -v parallel)" ]; then # put most time-consuming tests at the head of queue - time_consuming_tests="join_test.py hash_aggregate_test.py parquet_write_test.py" + time_consuming_tests="hash_aggregate_test.py join_test.py generate_expr_test.py parquet_write_test.py" tests_list=$(find "$SCRIPT_PATH"/src/main/python/ -name "*_test.py" -printf "%f ") tests=$(echo "$time_consuming_tests $tests_list" | tr ' ' '\n' | awk '!x[$0]++' | xargs) # --halt "now,fail=1": exit when the first job fail, and kill running jobs. From 4914be0d65ab5173ab597361560b81f083b2a241 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 8 Dec 2021 08:09:26 -0700 Subject: [PATCH 38/52] Fix typo in shim class name (#4310) Signed-off-by: Andy Grove --- .../scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala | 2 +- .../{Spark301util320Shims.scala => Spark301until320Shims.scala} | 2 +- .../scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) rename sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/{Spark301util320Shims.scala => Spark301until320Shims.scala} (99%) diff --git a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala index 78b50d670b5..ef680557167 100644 --- a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala +++ b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala @@ -47,7 +47,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManagerId} import org.apache.spark.unsafe.types.CalendarInterval -abstract class Spark30XShims extends Spark301util320Shims with Logging { +abstract class Spark30XShims extends Spark301until320Shims with Logging { override def int96ParquetRebaseRead(conf: SQLConf): String = parquetRebaseRead(conf) override def int96ParquetRebaseWrite(conf: SQLConf): String = diff --git a/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala b/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301until320Shims.scala similarity index 99% rename from sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala rename to sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301until320Shims.scala index c67af1d2972..d8a1ecbe60f 100644 --- a/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301util320Shims.scala +++ b/sql-plugin/src/main/301until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark301until320Shims.scala @@ -59,7 +59,7 @@ import org.apache.spark.sql.types._ /** * Shim base class that can be compiled with from 301 until 320 */ -trait Spark301util320Shims extends SparkShims { +trait Spark301until320Shims extends SparkShims { override def parquetRebaseReadKey: String = SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key override def parquetRebaseWriteKey: String = diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala index 0d2ec7c80ee..7b281c5dabd 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala @@ -50,7 +50,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManagerId} // 31x nondb shims, used by 311cdh and 31x -abstract class Spark31XShims extends Spark301util320Shims with Logging { +abstract class Spark31XShims extends Spark301until320Shims with Logging { override def int96ParquetRebaseRead(conf: SQLConf): String = conf.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ) From 61f51ffd6bf521f9c87451b04ec5a7d6011837f8 Mon Sep 17 00:00:00 2001 From: Rong Ou Date: Wed, 8 Dec 2021 12:43:23 -0800 Subject: [PATCH 39/52] Fix parquet options builder calls (#4331) Signed-off-by: Rong Ou --- .../scala/com/nvidia/spark/rapids/GpuParquetScanBase.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScanBase.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScanBase.scala index 34589cacae9..e94ea0e77ca 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScanBase.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScanBase.scala @@ -1100,7 +1100,6 @@ class MultiFileParquetPartitionReader( isSchemaCaseSensitive) val parseOpts = ParquetOptions.builder() .withTimeUnit(DType.TIMESTAMP_MICROSECONDS) - .enableStrictDecimalType(true) .includeColumn(includeColumns: _*).build() // About to start using the GPU @@ -1374,7 +1373,6 @@ class MultiFileCloudParquetPartitionReader( isSchemaCaseSensitive) val parseOpts = ParquetOptions.builder() .withTimeUnit(DType.TIMESTAMP_MICROSECONDS) - .enableStrictDecimalType(true) .includeColumn(includeColumns: _*).build() // about to start using the GPU @@ -1515,7 +1513,6 @@ class ParquetPartitionReader( isSchemaCaseSensitive) val parseOpts = ParquetOptions.builder() .withTimeUnit(DType.TIMESTAMP_MICROSECONDS) - .enableStrictDecimalType(true) .includeColumn(includeColumns: _*).build() // about to start using the GPU From c7483f2197741abc4d95e76d49c0bc5dfbf1b6b0 Mon Sep 17 00:00:00 2001 From: Alfred Xu Date: Thu, 9 Dec 2021 22:01:51 +0800 Subject: [PATCH 40/52] Support GpuFirst and GpuLast on nested types under reduction aggregations (#4337) Signed-off-by: sperlingxx --- docs/supported_ops.md | 24 +++++++------- .../src/main/python/hash_aggregate_test.py | 32 ++++++++----------- .../nvidia/spark/rapids/GpuOverrides.scala | 20 ++---------- 3 files changed, 27 insertions(+), 49 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 051c9a1906d..0597fe41d6f 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -14730,9 +14730,9 @@ are limited. S NS NS -NS -NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
NS @@ -14751,9 +14751,9 @@ are limited. S NS NS -NS -NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
NS @@ -14863,9 +14863,9 @@ are limited. S NS NS -NS -NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
NS @@ -14884,9 +14884,9 @@ are limited. S NS NS -NS -NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
NS diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 406ec0fb694..9d236adcf71 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -1008,28 +1008,22 @@ def test_count_distinct_with_nan_floats(data_gen): _nested_gens = array_gens_sample + struct_gens_sample + map_gens_sample @pytest.mark.parametrize('data_gen', decimal_gens + decimal_128_gens, ids=idfn) -def test_first_last_reductions_extra_types(data_gen): +def test_first_last_reductions_decimal_types(data_gen): assert_gpu_and_cpu_are_equal_collect( - # Coalesce and sort are to make sure that first and last, which are non-deterministic - # become deterministic - lambda spark : unary_op_df(spark, data_gen)\ - .coalesce(1).selectExpr( - 'first(a)', - 'last(a)'), - conf = allow_negative_scale_of_decimal_conf) + # Coalesce and sort are to make sure that first and last, which are non-deterministic + # become deterministic + lambda spark: unary_op_df(spark, data_gen).coalesce(1).selectExpr( + 'first(a)', 'last(a)', 'first(a, true)', 'last(a, true)'), + conf=allow_negative_scale_of_decimal_conf) -# TODO: https://github.com/NVIDIA/spark-rapids/issues/3221 -@allow_non_gpu('HashAggregateExec', 'SortAggregateExec', - 'ShuffleExchangeExec', 'HashPartitioning', - 'AggregateExpression', 'Alias', 'First', 'Last') @pytest.mark.parametrize('data_gen', _nested_gens, ids=idfn) -def test_first_last_reductions_nested_types_fallback(data_gen): - assert_cpu_and_gpu_are_equal_collect_with_capture( - lambda spark: unary_op_df(spark, data_gen, num_slices=1)\ - .selectExpr('first(a)', 'last(a)', 'first(a, True)', 'last(a, True)'), - exist_classes='First,Last', - non_exist_classes='GpuFirst,GpuLast', - conf=allow_negative_scale_of_decimal_conf) +def test_first_last_reductions_nested_types(data_gen): + assert_gpu_and_cpu_are_equal_collect( + # Coalesce and sort are to make sure that first and last, which are non-deterministic + # become deterministic + lambda spark: unary_op_df(spark, data_gen).coalesce(1).selectExpr( + 'first(a)', 'last(a)', 'first(a, true)', 'last(a, true)'), + conf=allow_negative_scale_of_decimal_conf) @pytest.mark.parametrize('data_gen', non_nan_all_basic_gens, ids=idfn) @pytest.mark.parametrize('parameterless', ['true', pytest.param('false', marks=pytest.mark.xfail( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index d51fdb531db..58174ba1ccf 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2294,14 +2294,7 @@ object GpuOverrides extends Logging { }), expr[First]( "first aggregate operator", { - val checks = ExprChecks.aggNotWindow( - TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL, TypeSig.all, - Seq(ParamCheck("input", - TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL, TypeSig.all)) - ).asInstanceOf[ExprChecksImpl] - // TODO: support GpuFirst on nested types for reduction - // https://github.com/NVIDIA/spark-rapids/issues/3221 - val nestedChecks = ContextChecks( + ExprChecks.aggNotWindow( (TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all, @@ -2310,7 +2303,6 @@ object GpuOverrides extends Logging { TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all)) ) - ExprChecksImpl(checks.contexts ++ Map(GroupByAggExprContext -> nestedChecks)) }, (a, conf, p, r) => new AggExprMeta[First](a, conf, p, r) { override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = @@ -2321,14 +2313,7 @@ object GpuOverrides extends Logging { }), expr[Last]( "last aggregate operator", { - val checks = ExprChecks.aggNotWindow( - TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL, TypeSig.all, - Seq(ParamCheck("input", - TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL, TypeSig.all)) - ).asInstanceOf[ExprChecksImpl] - // TODO: support GpuLast on nested types for reduction - // https://github.com/NVIDIA/spark-rapids/issues/3221 - val nestedChecks = ContextChecks( + ExprChecks.aggNotWindow( (TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all, @@ -2337,7 +2322,6 @@ object GpuOverrides extends Logging { TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all)) ) - ExprChecksImpl(checks.contexts ++ Map(GroupByAggExprContext -> nestedChecks)) }, (a, conf, p, r) => new AggExprMeta[Last](a, conf, p, r) { override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = From 59b9db1643bbd9e3e3e5c75cfe0e370597d52098 Mon Sep 17 00:00:00 2001 From: Alfred Xu Date: Thu, 9 Dec 2021 22:09:31 +0800 Subject: [PATCH 41/52] add test on writing empty table to ORC/PAQUET (#4333) Signed-off-by: sperlingxx --- .../src/main/python/orc_write_test.py | 12 ++++++++++ .../src/main/python/parquet_write_test.py | 22 +++++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py index 7b68c0a3f9d..859d15d3c74 100644 --- a/integration_tests/src/main/python/orc_write_test.py +++ b/integration_tests/src/main/python/orc_write_test.py @@ -152,3 +152,15 @@ def test_buckets_write_fallback(spark_tmp_path, spark_tmp_table_factory): data_path, 'DataWritingCommandExec', conf = {'spark.rapids.sql.format.orc.write.enabled': True}) + +@pytest.mark.parametrize('orc_gens', orc_write_gens_list, ids=idfn) +def test_write_empty_orc_round_trip(spark_tmp_path, orc_gens): + def create_empty_df(spark, path): + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] + return gen_df(spark, gen_list, length=0).write.orc(path) + data_path = spark_tmp_path + '/ORC_DATA' + assert_gpu_and_cpu_writes_are_equal_collect( + create_empty_df, + lambda spark, path: spark.read.orc(path), + data_path, + conf={'spark.rapids.sql.format.orc.write.enabled': True}) diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 7a5f9ff14e1..0709e5c3579 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -389,3 +389,25 @@ def test_it(spark): if allow_non_empty or e.desc.find('non-empty directory') == -1: raise e with_gpu_session(test_it, conf) + +@pytest.mark.parametrize('parquet_gens', parquet_write_gens_list, ids=idfn) +@pytest.mark.parametrize('reader_confs', reader_opt_confs) +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +@pytest.mark.parametrize('ts_type', parquet_ts_write_options) +def test_write_empty_parquet_round_trip(spark_tmp_path, + parquet_gens, + v1_enabled_list, + ts_type, + reader_confs): + def create_empty_df(spark, path): + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] + return gen_df(spark, gen_list, length=0).write.parquet(path) + data_path = spark_tmp_path + '/PARQUET_DATA' + all_confs = copy_and_update(reader_confs, writer_confs, { + 'spark.sql.sources.useV1SourceList': v1_enabled_list, + 'spark.sql.parquet.outputTimestampType': ts_type}) + assert_gpu_and_cpu_writes_are_equal_collect( + create_empty_df, + lambda spark, path: spark.read.parquet(path), + data_path, + conf=all_confs) From 1588f6a3d2a8cf78726ed7a5d871831079dfd8a5 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Thu, 9 Dec 2021 13:57:23 -0600 Subject: [PATCH 42/52] Allow returning an EmptyHashedRelation when a broadcast result is empty [databricks] (#4256) * Allow returning an EmptyHashedRelation when a broadcast result is empty Signed-off-by: Alessandro Bellina * Address review comments * Revert change in integration tests * Remove unwanted change * Minor cleanup * identity.length == 0 to identity.isEmpty Co-authored-by: Jason Lowe * Apply more suggested cleanup * Remove incRefCount * EmptyHashedRelation was introduced in 3.1.x, so this fixes the shims * Cache build schema outside of mapPartitions * Fix bug with the broadcast helper + make a new test in join_test * Adds a test that forces a broadcast for the EmptyHashedRelation scenario * Fix typo * Fix typo * Adding isFoldableNonLitAllowed to UnaryExprMeta * Fix Spark 3.0.x build * Also need to fix 30Xdb * Move isEmptyRelation override to Spark31xdb * Disable in databricks and do some clenaup * Parametrize databricks so we dont request AQE when that is an invalid config * Cleanup * Extra comment in RapidsMeta, take care of other review comments * Fix import spacing * Call the non-capturing assert * Apply suggestion in GpuBroadcastHelper Co-authored-by: Jason Lowe --- .../src/main/python/join_test.py | 27 +++++++ .../shims/v2/GpuBroadcastHashJoinExec.scala | 15 ++-- .../rapids/shims/v2/Spark30XdbShims.scala | 4 + .../shims/v2/GpuBroadcastHashJoinExec.scala | 15 ++-- .../spark/rapids/shims/v2/Spark30XShims.scala | 6 +- .../shims/v2/GpuBroadcastHashJoinExec.scala | 15 ++-- .../spark/rapids/shims/v2/Spark31XShims.scala | 14 +++- .../shims/v2/GpuBroadcastHashJoinExec.scala | 15 ++-- .../rapids/shims/v2/Spark31XdbShims.scala | 3 + .../spark/rapids/shims/v2/Spark32XShims.scala | 10 +++ .../com/nvidia/spark/rapids/RapidsMeta.scala | 10 +++ .../com/nvidia/spark/rapids/SparkShims.scala | 17 +++++ .../execution/GpuBroadcastExchangeExec.scala | 50 +++++++----- .../rapids/execution/GpuBroadcastHelper.scala | 76 +++++++++++++++++++ .../GpuBroadcastNestedLoopJoinExecBase.scala | 29 +++---- 15 files changed, 238 insertions(+), 68 deletions(-) create mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index 9a7ff888e26..ed7951d4baa 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -128,6 +128,33 @@ def do_join(spark): conf = copy_and_update(allow_negative_scale_of_decimal_conf, {'spark.rapids.sql.batchSizeBytes': batch_size}) assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf) +@ignore_order(local=True) +@pytest.mark.skipif(is_databricks_runtime(), + reason="Disabled for databricks because of lack of AQE support, and " + "differences in BroadcastMode.transform") +@pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn) +def test_right_broadcast_nested_loop_join_without_condition_empty_small_batch(join_type): + def do_join(spark): + left, right = create_df(spark, long_gen, 50, 0) + return left.join(broadcast(right), how=join_type) + conf = copy_and_update(allow_negative_scale_of_decimal_conf, + {'spark.sql.adaptive.enabled': 'true'}) + assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf) + +@ignore_order(local=True) +@pytest.mark.skipif(is_databricks_runtime(), + reason="Disabled for databricks because of lack of AQE support, and " + "differences in BroadcastMode.transform") +@pytest.mark.parametrize('join_type', ['Left', 'Right', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn) +def test_empty_broadcast_hash_join(join_type): + def do_join(spark): + left, right = create_df(spark, long_gen, 50, 0) + return left.join(right.hint("broadcast"), left.a == right.r_a, join_type) + conf = copy_and_update(allow_negative_scale_of_decimal_conf, + {'spark.sql.adaptive.enabled': 'true'}) + assert_gpu_and_cpu_are_equal_collect(do_join, conf = conf) + + # local sort because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) diff --git a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala index c1fed00bf28..ee3a0bea7dd 100644 --- a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuHashJoin, JoinTypeChecks, SerializeConcatHostBuffersDeserializeBatch} +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastHelper, GpuHashJoin, JoinTypeChecks} import org.apache.spark.sql.vectorized.ColumnarBatch class GpuBroadcastHashJoinMeta( @@ -148,16 +148,17 @@ case class GpuBroadcastHashJoinExec( val targetSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf) - val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + val broadcastRelation = broadcastExchange.executeColumnarBroadcast[Any]() val rdd = streamedPlan.executeColumnar() + val buildSchema = buildPlan.schema rdd.mapPartitions { it => val stIt = new CollectTimeIterator("broadcast join stream", it, streamTime) - val builtBatch = broadcastRelation.value.batch - GpuColumnVector.extractBases(builtBatch).foreach(_.noWarnLeakExpected()) - doJoin(builtBatch, stIt, targetSize, spillCallback, - numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) + withResource( + GpuBroadcastHelper.getBroadcastBatch(broadcastRelation, buildSchema)) { builtBatch => + doJoin(builtBatch, stIt, targetSize, spillCallback, + numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) + } } } } diff --git a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala index 39cbffeee55..63f18ae977f 100644 --- a/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala +++ b/sql-plugin/src/main/301db/scala/com/nvidia/spark/rapids/shims/v2/Spark30XdbShims.scala @@ -692,6 +692,10 @@ abstract class Spark30XdbShims extends Spark30XdbShimsBase with Logging { fileCatalog.allFiles().map(_.toFileStatus) } + // this is to help with an optimization in Spark 3.1, so we disable it by default in Spark 3.0.x + override def isEmptyRelation(relation: Any): Boolean = false + override def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] = None + override def broadcastModeTransform(mode: BroadcastMode, rows: Array[InternalRow]): Any = mode.transform(rows, TrampolineUtil.getTaskMemoryManager()) diff --git a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala index be6c29043bd..3f51de7a95e 100644 --- a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuHashJoin, JoinTypeChecks, SerializeConcatHostBuffersDeserializeBatch} +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastHelper, GpuHashJoin, JoinTypeChecks} import org.apache.spark.sql.vectorized.ColumnarBatch class GpuBroadcastHashJoinMeta( @@ -147,16 +147,17 @@ case class GpuBroadcastHashJoinExec( val targetSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf) - val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + val broadcastRelation = broadcastExchange.executeColumnarBroadcast[Any]() val rdd = streamedPlan.executeColumnar() + val buildSchema = buildPlan.schema rdd.mapPartitions { it => val stIt = new CollectTimeIterator("broadcast join stream", it, streamTime) - val builtBatch = broadcastRelation.value.batch - GpuColumnVector.extractBases(builtBatch).foreach(_.noWarnLeakExpected()) - doJoin(builtBatch, stIt, targetSize, spillCallback, - numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) + withResource( + GpuBroadcastHelper.getBroadcastBatch(broadcastRelation, buildSchema)) { builtBatch => + doJoin(builtBatch, stIt, targetSize, spillCallback, + numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) + } } } } diff --git a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala index ef680557167..d84c74e354c 100644 --- a/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala +++ b/sql-plugin/src/main/301until310-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark30XShims.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Average -import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} @@ -361,6 +361,10 @@ abstract class Spark30XShims extends Spark301until320Shims with Logging { adaptivePlan.initialPlan } + // this is to help with an optimization in Spark 3.1, so we disable it by default in Spark 3.0.x + override def isEmptyRelation(relation: Any): Boolean = false + override def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] = None + override def supportsColumnarAdaptivePlans: Boolean = false override def columnarAdaptivePlan(a: AdaptiveSparkPlanExec, goal: CoalesceSizeGoal): SparkPlan = { diff --git a/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala index bb81cad6083..46ecf9104be 100644 --- a/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuHashJoin, JoinTypeChecks, SerializeConcatHostBuffersDeserializeBatch} +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastHelper, GpuHashJoin, JoinTypeChecks} import org.apache.spark.sql.vectorized.ColumnarBatch class GpuBroadcastHashJoinMeta( @@ -149,16 +149,17 @@ case class GpuBroadcastHashJoinExec( val targetSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf) - val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + val broadcastRelation = broadcastExchange.executeColumnarBroadcast[Any]() val rdd = streamedPlan.executeColumnar() + val buildSchema = buildPlan.schema rdd.mapPartitions { it => val stIt = new CollectTimeIterator("broadcast join stream", it, streamTime) - val builtBatch = broadcastRelation.value.batch - GpuColumnVector.extractBases(builtBatch).foreach(_.noWarnLeakExpected()) - doJoin(builtBatch, stIt, targetSize, spillCallback, - numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) + withResource( + GpuBroadcastHelper.getBroadcastBatch(broadcastRelation, buildSchema)) { builtBatch => + doJoin(builtBatch, stIt, targetSize, spillCallback, + numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) + } } } } diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala index 7b281c5dabd..1af928e8c19 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala @@ -27,11 +27,12 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.rapids.shims.v2.GpuShuffleExchangeExec import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Average -import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} @@ -60,6 +61,17 @@ abstract class Spark31XShims extends Spark301until320Shims with Logging { SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ.key override def int96ParquetRebaseWriteKey: String = SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE.key + + override def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] = { + Some(broadcastModeTransform(mode, Array.empty)).filter(isEmptyRelation) + } + + override def isEmptyRelation(relation: Any): Boolean = relation match { + case EmptyHashedRelation => true + case arr: Array[InternalRow] if arr.isEmpty => true + case _ => false + } + override def hasSeparateINT96RebaseConf: Boolean = true override def getScalaUDFAsExpression( diff --git a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala index 7fbe789ff0f..fc3c55ffd53 100644 --- a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/GpuBroadcastHashJoinExec.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuHashJoin, JoinTypeChecks, SerializeConcatHostBuffersDeserializeBatch} +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastHelper, GpuHashJoin, JoinTypeChecks} import org.apache.spark.sql.vectorized.ColumnarBatch class GpuBroadcastHashJoinMeta( @@ -148,16 +148,17 @@ case class GpuBroadcastHashJoinExec( val targetSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf) - val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + val broadcastRelation = broadcastExchange.executeColumnarBroadcast[Any]() val rdd = streamedPlan.executeColumnar() + val buildSchema = buildPlan.schema rdd.mapPartitions { it => val stIt = new CollectTimeIterator("broadcast join stream", it, streamTime) - val builtBatch = broadcastRelation.value.batch - GpuColumnVector.extractBases(builtBatch).foreach(_.noWarnLeakExpected()) - doJoin(builtBatch, stIt, targetSize, spillCallback, - numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) + withResource( + GpuBroadcastHelper.getBroadcastBatch(broadcastRelation, buildSchema)) { builtBatch => + doJoin(builtBatch, stIt, targetSize, spillCallback, + numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) + } } } } diff --git a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala index 6649c12b498..7e8d11e8b36 100644 --- a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala +++ b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala @@ -831,6 +831,9 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { fileCatalog.allFiles().map(_.toFileStatus) } + override def isEmptyRelation(relation: Any): Boolean = false + override def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] = None + override def broadcastModeTransform(mode: BroadcastMode, rows: Array[InternalRow]): Any = mode.transform(rows, TaskContext.get.taskMemoryManager()) diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala index 452618d74b8..c4e159163e9 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala @@ -128,6 +128,16 @@ trait Spark32XShims extends SparkShims with Logging { fileIndex.allFiles() } + override def isEmptyRelation(relation: Any): Boolean = relation match { + case EmptyHashedRelation => true + case arr: Array[InternalRow] if arr.isEmpty => true + case _ => false + } + + override def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] = { + Some(broadcastModeTransform(mode, Array.empty)).filter(isEmptyRelation) + } + override final def broadcastModeTransform(mode: BroadcastMode, rows: Array[InternalRow]): Any = mode.transform(rows) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 37a72474e13..64a73ec8907 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1096,6 +1096,16 @@ abstract class UnaryExprMeta[INPUT <: UnaryExpression]( convertToGpu(childExprs.head.convertToGpu()) def convertToGpu(child: Expression): GpuExpression + + /** + * `ConstantFolding` executes early in the logical plan process, which + * simplifies many things before we get to the physical plan. If you enable + * AQE, some optimizations can cause new expressions to show up that would have been + * folded in by the logical plan optimizer (like `cast(null as bigint)` which just + * becomes Literal(null, Long) after `ConstantFolding`), so enabling this here + * allows us to handle these when they are generated by an AQE rule. + */ + override val isFoldableNonLitAllowed: Boolean = true } /** Base metadata class for unary expressions that support conversion to AST as well */ diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index d18bac8cf61..b8372e7392c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -253,8 +253,25 @@ trait SparkShims { def filesFromFileIndex(fileCatalog: PartitioningAwareFileIndex): Seq[FileStatus] + def isEmptyRelation(relation: Any): Boolean + def broadcastModeTransform(mode: BroadcastMode, toArray: Array[InternalRow]): Any + /** + * This call can produce an `EmptyHashedRelation` or an empty array, + * allowing the AQE rule `EliminateJoinToEmptyRelation` in Spark 3.1.x + * to optimize certain joins. + * + * In Spark 3.2.0, the optimization is still performed (under `AQEPropagateEmptyRelation`), + * but the AQE optimizer is looking at the metrics for the query stage to determine + * if numRows == 0, and if so it can eliminate certain joins. + * + * The call is implemented only for Spark 3.1.x+. It is disabled in + * Databricks because it requires a task context to perform the + * `BroadcastMode.transform` call, but we'd like to call this from the driver. + */ + def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] + def isAqePlan(p: SparkPlan): Boolean def isExchangeOp(plan: SparkPlanMeta[_]): Boolean diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala index ceede1fd0cc..e697d41ac34 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala @@ -303,28 +303,36 @@ abstract class GpuBroadcastExchangeExecBase( // Setup a job group here so later it may get cancelled by groupId if necessary. sparkContext.setJobGroup(_runId.toString, s"broadcast exchange (runId ${_runId})", interruptOnCancel = true) - val batch = withResource(new NvtxWithMetrics("broadcast collect", NvtxColor.GREEN, - collectTime)) { _ => - val data = child.executeColumnar().map(cb => try { - new SerializeBatchDeserializeHostBuffer(cb) - } finally { - cb.close() - }) - val d = data.collect() - new SerializeConcatHostBuffersDeserializeBatch(d, output) - } - - val numRows = batch.numRows - checkRowLimit(numRows) - numOutputBatches += 1 - numOutputRows += numRows - + var dataSize = 0L + val broadcastResult = + withResource(new NvtxWithMetrics("broadcast collect", NvtxColor.GREEN, + collectTime)) { _ => + val childRdd = child.executeColumnar() + val data = childRdd.map(cb => try { + new SerializeBatchDeserializeHostBuffer(cb) + } finally { + cb.close() + }) + + val d = data.collect() + val emptyRelation: Option[Any] = if (d.isEmpty) { + ShimLoader.getSparkShims.tryTransformIfEmptyRelation(mode) + } else { + None + } + + emptyRelation.getOrElse({ + val batch = new SerializeConcatHostBuffersDeserializeBatch(d, output) + val numRows = batch.numRows + checkRowLimit(numRows) + numOutputBatches += 1 + numOutputRows += numRows + dataSize += batch.dataSize + batch + }) + } withResource(new NvtxWithMetrics("broadcast build", NvtxColor.DARK_GREEN, buildTime)) { _ => - // we only support hashjoin so this is a noop - // val relation = mode.transform(input, Some(numRows)) - val dataSize = batch.dataSize - gpuLongMetric("dataSize") += dataSize if (dataSize >= MAX_BROADCAST_TABLE_BYTES) { throw new SparkException( @@ -335,7 +343,7 @@ abstract class GpuBroadcastExchangeExecBase( val broadcasted = withResource(new NvtxWithMetrics("broadcast", NvtxColor.CYAN, broadcastTime)) { _ => // Broadcast the relation - sparkContext.broadcast(batch.asInstanceOf[Any]) + sparkContext.broadcast(broadcastResult) } SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala new file mode 100644 index 00000000000..47b9c1cada2 --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala @@ -0,0 +1,76 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.execution + +import com.nvidia.spark.rapids.{GpuColumnVector, ShimLoader} + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +object GpuBroadcastHelper { + /** + * Given a broadcast relation get a ColumnarBatch that can be used on the GPU. + * + * The broadcast relation may or may not contain any data, so we special case + * the empty relation case (hash or identity depending on the type of join). + * + * If a broadcast result is unexpected we throw, but at this moment other + * cases are not known, so this is a defensive measure. + * + * @param broadcastRelation - the broadcast as produced by a broadcast exchange + * @param broadcastSchema - the broadcast schema + * @return a `ColumnarBatch` or throw if the broadcast can't be handled + */ + def getBroadcastBatch(broadcastRelation: Broadcast[Any], + broadcastSchema: StructType): ColumnarBatch = { + broadcastRelation.value match { + case broadcastBatch: SerializeConcatHostBuffersDeserializeBatch => + val builtBatch = broadcastBatch.batch + GpuColumnVector.incRefCounts(builtBatch) + builtBatch + case v if ShimLoader.getSparkShims.isEmptyRelation(v) => + GpuColumnVector.emptyBatch(broadcastSchema) + case t => + throw new IllegalStateException(s"Invalid broadcast batch received $t") + } + } + + /** + * Given a broadcast relation get the number of rows that the received batch + * contains + * + * The broadcast relation may or may not contain any data, so we special case + * the empty relation case (hash or identity depending on the type of join). + * + * If a broadcast result is unexpected we throw, but at this moment other + * cases are not known, so this is a defensive measure. + * + * @param broadcastRelation - the broadcast as produced by a broadcast exchange + * @return number of rows for a batch received, or 0 if it's an empty relation + */ + def getBroadcastBatchNumRows(broadcastRelation: Broadcast[Any]): Int = { + broadcastRelation.value match { + case broadcastBatch: SerializeConcatHostBuffersDeserializeBatch => + broadcastBatch.batch.numRows() + case v if ShimLoader.getSparkShims.isEmptyRelation(v) => 0 + case t => + throw new IllegalStateException(s"Invalid broadcast batch received $t") + } + } +} + diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala index dac5429f07f..e13c69479c1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala @@ -435,23 +435,23 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } private[this] def makeBuiltBatch( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], + broadcastRelation: Broadcast[Any], buildTime: GpuMetric, buildDataSize: GpuMetric): ColumnarBatch = { withResource(new NvtxWithMetrics("build join table", NvtxColor.GREEN, buildTime)) { _ => - val ret = broadcastRelation.value.batch - buildDataSize += GpuColumnVector.getTotalDeviceMemoryUsed(ret) - GpuColumnVector.incRefCounts(ret) + val builtBatch = GpuBroadcastHelper.getBroadcastBatch(broadcastRelation, broadcast.schema) + buildDataSize += GpuColumnVector.getTotalDeviceMemoryUsed(builtBatch) + builtBatch } } private[this] def computeBuildRowCount( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], + broadcastRelation: Broadcast[Any], buildTime: GpuMetric, buildDataSize: GpuMetric): Int = { withResource(new NvtxWithMetrics("build join table", NvtxColor.GREEN, buildTime)) { _ => buildDataSize += 0 - broadcastRelation.value.batch.numRows() + GpuBroadcastHelper.getBroadcastBatchNumRows(broadcastRelation) } } @@ -468,7 +468,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } val broadcastRelation = - broadcastExchange.executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + broadcastExchange.executeColumnarBroadcast[Any]() val joinCondition = boundCondition.orElse { // For outer joins use a true condition if there are any columns in the build side @@ -489,7 +489,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } private def leftExistenceJoin( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], + broadcastRelation: Broadcast[Any], exists: Boolean, buildTime: GpuMetric, buildDataSize: GpuMetric): RDD[ColumnarBatch] = { @@ -504,9 +504,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } } - private def doUnconditionalJoin( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch] - ): RDD[ColumnarBatch] = { + private def doUnconditionalJoin(broadcastRelation: Broadcast[Any]): RDD[ColumnarBatch] = { if (output.isEmpty) { doUnconditionalJoinRowCount(broadcastRelation) } else { @@ -565,9 +563,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } /** Special-case handling of an unconditional join that just needs to output a row count. */ - private def doUnconditionalJoinRowCount( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch] - ): RDD[ColumnarBatch] = { + private def doUnconditionalJoinRowCount(broadcastRelation: Broadcast[Any]): RDD[ColumnarBatch] = { if (joinType == LeftAnti) { // degenerate case, no rows are returned. left.executeColumnar().mapPartitions { _ => @@ -604,13 +600,13 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } private def doConditionalJoin( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], + broadcastRelation: Broadcast[Any], boundCondition: Option[GpuExpression], numFirstTableColumns: Int): RDD[ColumnarBatch] = { val buildTime = gpuLongMetric(BUILD_TIME) val buildDataSize = gpuLongMetric(BUILD_DATA_SIZE) + val spillCallback = GpuMetric.makeSpillCallback(allMetrics) lazy val builtBatch = makeBuiltBatch(broadcastRelation, buildTime, buildDataSize) - val streamAttributes = streamed.output val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) @@ -619,7 +615,6 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val nestedLoopJoinType = joinType val buildSide = getGpuBuildSide - val spillCallback = GpuMetric.makeSpillCallback(allMetrics) streamed.executeColumnar().mapPartitions { streamedIter => val lazyStream = streamedIter.map { cb => withResource(cb) { cb => From f99a57bdbe99c01e47e549e92a80ca00b23427fb Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Fri, 10 Dec 2021 12:24:17 +0800 Subject: [PATCH 43/52] Remove the unnecessary parameter dataType in resolveColumnVector method (#4340) Signed-off-by: Firestarman --- .../scala/com/nvidia/spark/rapids/GpuExpandExec.scala | 2 +- .../com/nvidia/spark/rapids/GpuExpressions.scala | 11 +++++------ 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala index 4574f5ba14e..2e4730cc45c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala @@ -163,7 +163,7 @@ class GpuExpandIterator( val (cv, nullColumnReused) = expr.columnarEval(cb) match { case null => getOrCreateNullCV(sparkType) case other => - (GpuExpressionsUtils.resolveColumnVector(other, cb.numRows, sparkType), false) + (GpuExpressionsUtils.resolveColumnVector(other, cb.numRows), false) } if (!nullColumnReused) { uniqueDeviceColumns += cv diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala index c2a171f7b46..7732f76bbc4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala @@ -23,7 +23,7 @@ import com.nvidia.spark.rapids.shims.v2.{ShimBinaryExpression, ShimExpression, S import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.sql.types.StringType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.UTF8String @@ -64,15 +64,14 @@ object GpuExpressionsUtils extends Arm { * should cover all the cases for GPU pipelines. * * @param any the input value. It will be closed if it is a closeable after the call done. - * @param numRows the expected row number of the output column, used when 'any' is a Scalar. - * @param dType the data type of the output column, used when 'any' is a Scalar. + * @param numRows the expected row number of the output column, used when 'any' is a GpuScalar. * @return a `GpuColumnVector` if it succeeds. Users should close the column vector to avoid * memory leak. */ - def resolveColumnVector(any: Any, numRows: Int, dType: DataType): GpuColumnVector = { + def resolveColumnVector(any: Any, numRows: Int): GpuColumnVector = { withResourceIfAllowed(any) { case c: GpuColumnVector => c.incRefCount() - case s: GpuScalar => GpuColumnVector.from(s, numRows, dType) + case s: GpuScalar => GpuColumnVector.from(s, numRows, s.dataType) case other => throw new IllegalArgumentException(s"Cannot resolve a ColumnVector from the value:" + s" $other. Please convert it to a GpuScalar or a GpuColumnVector before returning.") @@ -91,7 +90,7 @@ object GpuExpressionsUtils extends Arm { * memory leak. */ def columnarEvalToColumn(expr: Expression, batch: ColumnarBatch): GpuColumnVector = - resolveColumnVector(expr.columnarEval(batch), batch.numRows, expr.dataType) + resolveColumnVector(expr.columnarEval(batch), batch.numRows) /** * Extract the GpuLiteral From 024e5cb64e1bd218dc96609d301c8081181564bb Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Fri, 10 Dec 2021 07:07:59 -0600 Subject: [PATCH 44/52] Use scala Int.box instead of Integer constructors (#4338) * Use scala Int.box instead Integer constructors which are deprecated in JDK11 Signed-off-by: Alessandro Bellina * Apply code review suggestion Co-authored-by: Gera Shegalov * Revert incorrect suggestion Signed-off-by: Gera Shegalov * Brings back Int.box Co-authored-by: Gera Shegalov Co-authored-by: Gera Shegalov --- .../nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala index f1c4ca1a7d2..d881624c05d 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala @@ -165,7 +165,7 @@ object RapidsShuffleTestHelper extends MockitoSugar with Arm { } def withMockContiguousTable[T](numRows: Long)(body: ContiguousTable => T): T = { - val rows: Seq[Integer] = (0 until numRows.toInt).map(new Integer(_)) + val rows: Seq[Integer] = (0 until numRows.toInt).map(Int.box) withResource(ColumnVector.fromBoxedInts(rows:_*)) { cvBase => cvBase.incRefCount() val gpuCv = GpuColumnVector.from(cvBase, IntegerType) From bdfc6ce0d2a58ccbd0d6f5fdd15b9b75ca3bac23 Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Sat, 11 Dec 2021 09:13:03 +0800 Subject: [PATCH 45/52] Change the explanation of why the operator will not work on GPU (#4328) * Change the explanation of log massage when an operator cannot be supported by GPU. Signed-off-by: remzi <13716567376yh@gmail.com> --- .../getting-started-workload-qualification.md | 4 ++-- integration_tests/src/main/python/explain_test.py | 2 +- .../scala/com/nvidia/spark/rapids/RapidsMeta.scala | 12 ++++++------ 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/get-started/getting-started-workload-qualification.md b/docs/get-started/getting-started-workload-qualification.md index 340dd84e5f9..02688577cb8 100644 --- a/docs/get-started/getting-started-workload-qualification.md +++ b/docs/get-started/getting-started-workload-qualification.md @@ -124,7 +124,7 @@ the driver logs with `spark.rapids.sql.explain=all`. this version: ``` - !NOT_FOUND cannot run on GPU because no GPU enabled version of operator class org.apache.spark.sql.execution.RowDataSourceScanExec could be found + ! cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RowDataSourceScanExec ``` This log can show you which operators (on what data type) can not run on GPU and the reason. @@ -152,7 +152,7 @@ analysis. For example, the log lines starting with `!` is the so-called not-supported messages: ``` !Exec cannot run on GPU because not all expressions can be replaced - !NOT_FOUND replicaterows(sum#99L, gender#76) cannot run on GPU because no GPU enabled version of expression class + ! replicaterows(sum#99L, gender#76) cannot run on GPU because GPU does not currently support the operator ReplicateRows ``` The indentation indicates the parent and child relationship for those expressions. If not all of the children expressions can run on GPU, the parent can not run on GPU either. diff --git a/integration_tests/src/main/python/explain_test.py b/integration_tests/src/main/python/explain_test.py index 597df7b9bbd..9c56cf5c340 100644 --- a/integration_tests/src/main/python/explain_test.py +++ b/integration_tests/src/main/python/explain_test.py @@ -83,7 +83,7 @@ def do_explain(spark): df2 = df.select(slen("name").alias("slen(name)"), to_upper("name"), add_one("age")) explain_str = spark.sparkContext._jvm.com.nvidia.spark.rapids.ExplainPlan.explainPotentialGpuPlan(df2._jdf, "ALL") # udf shouldn't be on GPU - udf_str_not = 'cannot run on GPU because no GPU enabled version of operator class org.apache.spark.sql.execution.python.BatchEvalPythonExec' + udf_str_not = 'cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.python.BatchEvalPythonExec' assert udf_str_not in explain_str not_on_gpu_str = spark.sparkContext._jvm.com.nvidia.spark.rapids.ExplainPlan.explainPotentialGpuPlan(df2._jdf, "NOT") assert udf_str_not in not_on_gpu_str diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 64a73ec8907..ff5d159c190 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -46,7 +46,7 @@ trait DataFromReplacementRule { * A version of DataFromReplacementRule that is used when no replacement rule can be found. */ final class NoRuleDataFromReplacementRule extends DataFromReplacementRule { - override val operationName: String = "NOT_FOUND" + override val operationName: String = "" override def confKey = "NOT_FOUND" @@ -463,7 +463,7 @@ final class RuleNotFoundPartMeta[INPUT <: Partitioning]( extends PartMeta[INPUT](part, conf, parent, new NoRuleDataFromReplacementRule) { override def tagPartForGpu(): Unit = { - willNotWorkOnGpu(s"no GPU enabled version of partitioning ${part.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${part.getClass}") } override def convertToGpu(): GpuPartitioning = @@ -498,7 +498,7 @@ final class RuleNotFoundScanMeta[INPUT <: Scan]( extends ScanMeta[INPUT](scan, conf, parent, new NoRuleDataFromReplacementRule) { override def tagSelfForGpu(): Unit = { - willNotWorkOnGpu(s"no GPU enabled version of scan ${scan.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${scan.getClass}") } override def convertToGpu(): Scan = @@ -534,7 +534,7 @@ final class RuleNotFoundDataWritingCommandMeta[INPUT <: DataWritingCommand]( extends DataWritingCommandMeta[INPUT](cmd, conf, parent, new NoRuleDataFromReplacementRule) { override def tagSelfForGpu(): Unit = { - willNotWorkOnGpu(s"no GPU accelerated version of command ${cmd.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${cmd.getClass}") } override def convertToGpu(): GpuDataWritingCommand = @@ -795,7 +795,7 @@ final class RuleNotFoundSparkPlanMeta[INPUT <: SparkPlan]( extends SparkPlanMeta[INPUT](plan, conf, parent, new NoRuleDataFromReplacementRule) { override def tagPlanForGpu(): Unit = - willNotWorkOnGpu(s"no GPU enabled version of operator ${plan.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${plan.getClass}") override def convertToGpu(): GpuExec = throw new IllegalStateException("Cannot be converted to GPU") @@ -1317,7 +1317,7 @@ final class RuleNotFoundExprMeta[INPUT <: Expression]( extends ExprMeta[INPUT](expr, conf, parent, new NoRuleDataFromReplacementRule) { override def tagExprForGpu(): Unit = - willNotWorkOnGpu(s"no GPU enabled version of expression ${expr.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${expr.getClass}") override def convertToGpu(): GpuExpression = throw new IllegalStateException("Cannot be converted to GPU") From bc0cccba5d44c4983d606648cda4a164fe8cdafa Mon Sep 17 00:00:00 2001 From: Peixin Date: Mon, 13 Dec 2021 09:02:03 +0800 Subject: [PATCH 46/52] Split expensive pytest files in cases level [skip ci] (#4336) * Split expensive pytest files in cases level Signed-off-by: Peixin Li * add more doc and rename func --- jenkins/spark-tests.sh | 72 ++++++++++++++++++++++++++++++------------ 1 file changed, 52 insertions(+), 20 deletions(-) diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index 1387be78d66..be76e0978ba 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -114,7 +114,9 @@ export SPARK_TASK_MAXFAILURES=1 [[ "$IS_SPARK_311_OR_LATER" -eq "0" ]] && SPARK_TASK_MAXFAILURES=4 export PATH="$SPARK_HOME/bin:$SPARK_HOME/sbin:$PATH" - +# enable worker cleanup to avoid "out of space" issue +# if failed, we abort the test instantly, so the failed executor log should still be left there for debugging +export SPARK_WORKER_OPTS="$SPARK_WORKER_OPTS -Dspark.worker.cleanup.enabled=true -Dspark.worker.cleanup.interval=120 -Dspark.worker.cleanup.appDataTtl=60" #stop and restart SPARK ETL stop-slave.sh stop-master.sh @@ -137,18 +139,12 @@ export BASE_SPARK_SUBMIT_ARGS="$BASE_SPARK_SUBMIT_ARGS \ export SEQ_CONF="--executor-memory 16G \ --total-executor-cores 6" -# currently we hardcode the parallelism and configs based on our CI node's hardware specs, -# we can make it dynamically generated if this script is going to be used in other scenarios in the future -PARALLELISM=${PARALLELISM:-'4'} -MEMORY_FRACTION=$(python -c "print(1/($PARALLELISM + 0.2))") export PARALLEL_CONF="--executor-memory 4G \ --total-executor-cores 1 \ --conf spark.executor.cores=1 \ --conf spark.task.cpus=1 \ --conf spark.rapids.sql.concurrentGpuTasks=1 \ ---conf spark.rapids.memory.gpu.minAllocFraction=0 \ ---conf spark.rapids.memory.gpu.allocFraction=${MEMORY_FRACTION} \ ---conf spark.rapids.memory.gpu.maxAllocFraction=${MEMORY_FRACTION}" +--conf spark.rapids.memory.gpu.minAllocFraction=0" export CUDF_UDF_TEST_ARGS="--conf spark.rapids.memory.gpu.allocFraction=0.1 \ --conf spark.rapids.memory.gpu.minAllocFraction=0 \ @@ -165,7 +161,7 @@ export SCRIPT_PATH="$(pwd -P)" export TARGET_DIR="$SCRIPT_PATH/target" mkdir -p $TARGET_DIR -run_test() { +run_test_not_parallel() { local TEST=${1//\.py/} local LOG_FILE case $TEST in @@ -190,7 +186,7 @@ run_test() { LOG_FILE="$TARGET_DIR/$TEST.log" # set dedicated RUN_DIRs here to avoid conflict between parallel tests RUN_DIR="$TARGET_DIR/run_dir_$TEST" \ - SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $PARALLEL_CONF" \ + SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $PARALLEL_CONF $MEMORY_FRACTION_CONF" \ ./run_pyspark_from_build.sh -k $TEST >"$LOG_FILE" 2>&1 CODE="$?" @@ -204,7 +200,7 @@ run_test() { ;; esac } -export -f run_test +export -f run_test_not_parallel # TEST_MODE # - IT_ONLY @@ -214,27 +210,63 @@ TEST_MODE=${TEST_MODE:-'IT_ONLY'} if [[ $TEST_MODE == "ALL" || $TEST_MODE == "IT_ONLY" ]]; then # integration tests if [[ $PARALLEL_TEST == "true" ]] && [ -x "$(command -v parallel)" ]; then - # put most time-consuming tests at the head of queue - time_consuming_tests="hash_aggregate_test.py join_test.py generate_expr_test.py parquet_write_test.py" - tests_list=$(find "$SCRIPT_PATH"/src/main/python/ -name "*_test.py" -printf "%f ") - tests=$(echo "$time_consuming_tests $tests_list" | tr ' ' '\n' | awk '!x[$0]++' | xargs) + # We separate tests/cases into different categories for parallel run to try avoid long tail distribution + # time_consuming_tests: tests that would cost over 1 hour if run sequentially, we split them into cases (time_consuming_tests_cases) + # mem_consuming_cases: cases in time_consuming_tests that would consume much more GPU memory than normal cases + # other_tests: tests except time_consuming_tests_cases and mem_consuming_cases + + # TODO: Tag these tests/cases + # time-consuming tests, space-separated + time_consuming_tests="join_test hash_aggregate_test generate_expr_test parquet_write_test orc_test orc_write_test" + # GPU memory-consuming cases in time_consuming_tests, space-separated + mem_consuming_cases="test_hash_reduction_decimal_overflow_sum" + # hardcode parallelism as 2 for gpu-mem consuming cases + export MEMORY_FRACTION_CONF="--conf spark.rapids.memory.gpu.allocFraction=0.45 \ + --conf spark.rapids.memory.gpu.maxAllocFraction=0.45" # --halt "now,fail=1": exit when the first job fail, and kill running jobs. # we can set it to "never" and print failed ones after finish running all tests if needed # --group: print stderr after test finished for better readability - parallel --group --halt "now,fail=1" -j"${PARALLELISM}" run_test ::: $tests + parallel --group --halt "now,fail=1" -j2 run_test_not_parallel ::: ${mem_consuming_cases} + + time_consuming_tests_str=$(echo ${time_consuming_tests} | xargs | sed 's/ / or /g') + mem_consuming_cases_str=$(echo ${mem_consuming_cases} | xargs | sed 's/ / and not /g') + time_consuming_tests_cases=$(./run_pyspark_from_build.sh -k \ + "(${time_consuming_tests_str}) and not ${mem_consuming_cases_str}" \ + --collect-only -qq 2>/dev/null | grep -oP '(?<=::).*?(?=\[)' | uniq | shuf | xargs) + other_tests=$(./run_pyspark_from_build.sh --collect-only -qqq 2>/dev/null | grep -oP '(?<=python/).*?(?=.py)' \ + | grep -vP "$(echo ${time_consuming_tests} | xargs | tr ' ' '|')") + tests=$(echo "${time_consuming_tests_cases} ${other_tests}" | tr ' ' '\n' | awk '!x[$0]++' | xargs) + + if [[ "${PARALLELISM}" == "" ]]; then + PARALLELISM=$(nvidia-smi --query-gpu=memory.free --format=csv,noheader | \ + awk '{if (MAX < $1){ MAX = $1}} END {print int(MAX / (2 * 1024))}') + fi + MEMORY_FRACTION=$(python -c "print(1/($PARALLELISM + 0.1))") + export MEMORY_FRACTION_CONF="--conf spark.rapids.memory.gpu.allocFraction=${MEMORY_FRACTION} \ + --conf spark.rapids.memory.gpu.maxAllocFraction=${MEMORY_FRACTION}" + parallel --group --halt "now,fail=1" -j"${PARALLELISM}" run_test_not_parallel ::: $tests else - run_test all + run_test_not_parallel all fi - # Temporarily only run on Spark 3.1.1 (https://github.com/NVIDIA/spark-rapids/issues/3311) if [[ "$IS_SPARK_311_OR_LATER" -eq "1" ]]; then - run_test cache_serializer + if [[ $PARALLEL_TEST == "true" ]] && [ -x "$(command -v parallel)" ]; then + cache_test_cases=$(./run_pyspark_from_build.sh -k "cache_test" \ + --collect-only -qq 2>/dev/null | grep -oP '(?<=::).*?(?=\[)' | uniq | shuf | xargs) + # hardcode parallelism as 5 + export MEMORY_FRACTION_CONF="--conf spark.rapids.memory.gpu.allocFraction=0.18 \ + --conf spark.rapids.memory.gpu.maxAllocFraction=0.18 \ + --conf spark.sql.cache.serializer=com.nvidia.spark.ParquetCachedBatchSerializer" + parallel --group --halt "now,fail=1" -j5 run_test_not_parallel ::: ${cache_test_cases} + else + run_test_not_parallel cache_serializer + fi fi fi # cudf_udf_test if [[ "$TEST_MODE" == "ALL" || "$TEST_MODE" == "CUDF_UDF_ONLY" ]]; then - run_test cudf_udf_test + run_test_not_parallel cudf_udf_test fi popd From dab062eb2c7385a8488cc42d7c795b46cba8f94c Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 13 Dec 2021 13:45:34 +0800 Subject: [PATCH 47/52] Fix the issue of exporting Column RDD (#4335) Signed-off-by: Bobby Wang --- .../spark/rapids/shims/v2/Spark31XShims.scala | 4 ++-- .../spark/rapids/shims/v2/Spark31XdbShims.scala | 4 ++-- .../spark/rapids/shims/v2/Spark32XShims.scala | 4 ++-- .../execution/InternalColumnarRddConverter.scala | 8 +++++++- .../InternalColumnarRDDConverterSuite.scala | 14 +++++++++++++- 5 files changed, 26 insertions(+), 8 deletions(-) diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala index 1af928e8c19..66e63bee45a 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XShims.scala @@ -466,9 +466,9 @@ abstract class Spark31XShims extends Spark301until320Shims with Logging { val serName = plan.conf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) val serClass = ShimLoader.loadClass(serName) if (serClass == classOf[com.nvidia.spark.ParquetCachedBatchSerializer]) { - GpuColumnarToRowTransitionExec(plan) + GpuColumnarToRowTransitionExec(plan, exportColumnRdd) } else { - GpuColumnarToRowExec(plan) + GpuColumnarToRowExec(plan, exportColumnRdd) } } diff --git a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala index 7e8d11e8b36..f501545c5c5 100644 --- a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala +++ b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/v2/Spark31XdbShims.scala @@ -636,9 +636,9 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { val serName = plan.conf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) val serClass = ShimLoader.loadClass(serName) if (serClass == classOf[com.nvidia.spark.ParquetCachedBatchSerializer]) { - GpuColumnarToRowTransitionExec(plan) + GpuColumnarToRowTransitionExec(plan, exportColumnRdd) } else { - GpuColumnarToRowExec(plan) + GpuColumnarToRowExec(plan, exportColumnRdd) } } diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala index c4e159163e9..a5dac20c59d 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/v2/Spark32XShims.scala @@ -866,9 +866,9 @@ trait Spark32XShims extends SparkShims with Logging { val serName = plan.conf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) val serClass = ShimLoader.loadClass(serName) if (serClass == classOf[com.nvidia.spark.ParquetCachedBatchSerializer]) { - GpuColumnarToRowTransitionExec(plan) + GpuColumnarToRowTransitionExec(plan, exportColumnRdd) } else { - GpuColumnarToRowExec(plan) + GpuColumnarToRowExec(plan, exportColumnRdd) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala index af5a46d3fac..3a51b5d89a1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala @@ -654,7 +654,8 @@ object InternalColumnarRddConverter extends Logging { convert(df) } - def convert(df: DataFrame): RDD[Table] = { + // Extract RDD[ColumnarBatch] directly + def extractRDDColumnarBatch(df: DataFrame): (Option[RDD[ColumnarBatch]], RDD[Row]) = { val schema = df.schema val unsupported = schema.map(_.dataType).filter( dt => !GpuOverrides.isSupportedType(dt, allowMaps = true, allowStringMaps = true, allowNull = true, allowStruct = true, allowArray @@ -709,7 +710,12 @@ object InternalColumnarRddConverter extends Logging { logDebug(s"Cannot extract columnar RDD directly. " + s"(First MapPartitionsRDD not found $rdd)") } + (batch, input) + } + def convert(df: DataFrame): RDD[Table] = { + val schema = df.schema + val (batch, input) = extractRDDColumnarBatch(df) val b = batch.getOrElse({ // We have to fall back to doing a slow transition. val converters = new GpuExternalRowToColumnConverter(schema) diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala index 7ce1d9fdc4f..89d3ccb743a 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala @@ -18,9 +18,10 @@ package org.apache.spark.sql.rapids.execution import scala.collection.mutable -import com.nvidia.spark.rapids.{ColumnarToRowIterator, GpuBatchUtilsSuite, NoopMetric, SparkQueryCompareTestSuite} +import com.nvidia.spark.rapids.{ColumnarToRowIterator, GpuBatchUtilsSuite, NoopMetric, SparkQueryCompareTestSuite, TestResourceFinder} import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder +import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.util.MapData import org.apache.spark.sql.types._ @@ -272,5 +273,16 @@ class InternalColumnarRDDConverterSuite extends SparkQueryCompareTestSuite { } } + test("InternalColumnarRddConverter should extractRDDTable RDD[ColumnarBatch]") { + withGpuSparkSession(spark => { + val path = TestResourceFinder.getResourcePath("disorder-read-schema.parquet") + val df = spark.read.parquet(path) + val (optionRddColumnBatch, _) = InternalColumnarRddConverter.extractRDDColumnarBatch(df) + + assert(optionRddColumnBatch.isDefined, "Can't extract RDD[ColumnarBatch]") + + }, new SparkConf().set("spark.rapids.sql.test.allowedNonGpu", "DeserializeToObjectExec")) + } + } From 7f5e2b3982c004900f5735767f3b04c9ae476cbd Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Mon, 13 Dec 2021 15:04:04 +0800 Subject: [PATCH 48/52] The option for the nightly script not deploying jars [skip ci] (#4269) * The option for the nightly script not deploying jars [skip ci] The nightly script will build but not deploy jars if set SKIP_DEPLOY="true". This can be used when testing updates to the scripts, as we only need to verify whether the updates work for the nightly build but do not need to deploy jars. Signed-off-by: Tim Liu * Default value for var SKIP_DEPLOY * Condition check using [[ != 'true' ]] to match val's description * Condition check using if [[ != 'true' ]] Signed-off-by: Tim Liu * unnecessary after the condition reworked to [[ != 'true' ]] Signed-off-by: Tim Liu --- jenkins/spark-nightly-build.sh | 27 ++++++++++++++++----------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/jenkins/spark-nightly-build.sh b/jenkins/spark-nightly-build.sh index c301a5d844b..d353e0da878 100755 --- a/jenkins/spark-nightly-build.sh +++ b/jenkins/spark-nightly-build.sh @@ -24,29 +24,34 @@ export M2DIR="$WORKSPACE/.m2" # build, install, and deploy all the versions we support, but skip deploy of individual dist module since we # only want the combined jar to be pushed. # Note this does not run any integration tests +# Deploy jars unless SKIP_DEPLOY is 'true' mvn -U -B -Dbuildver=302 clean install -pl '!tools' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -mvn -B -Dbuildver=302 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=302 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests mvn -U -B -Dbuildver=303 clean install -pl '!tools' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -mvn -B -Dbuildver=303 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=303 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests mvn -U -B -Dbuildver=304 clean install -pl '!tools' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -mvn -B -Dbuildver=304 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=304 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests mvn -U -B -Dbuildver=311 clean install $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -mvn -B -Dbuildver=311 deploy -pl -dist $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=311 deploy -pl -dist $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests mvn -U -B -Dbuildver=312 clean install -pl '!tools' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -mvn -B -Dbuildver=312 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=312 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests mvn -U -B -Dbuildver=313 clean install -pl '!tools' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -mvn -B -Dbuildver=313 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=313 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests mvn -U -B -Dbuildver=311cdh clean install -pl '!tools' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -mvn -B -Dbuildver=311cdh deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=311cdh deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests mvn -U -B -Dbuildver=320 clean install -pl '!tools' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -mvn -B -Dbuildver=320 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=320 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests mvn -U -B -Dbuildver=321 clean install -pl '!tools' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -mvn -B -Dbuildver=321 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=321 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests # temporarily skip tests on Spark 3.3.0 - https://github.com/NVIDIA/spark-rapids/issues/4031 mvn -U -B -Dbuildver=330 clean install -pl '!tools' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests -mvn -B -Dbuildver=330 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests +[[ $SKIP_DEPLOY != 'true' ]] && mvn -B -Dbuildver=330 deploy -pl '!tools,!dist' $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER -DskipTests -mvn -B -pl '!tools' -Dbuildver=301 -PsnapshotsWithDatabricks clean deploy $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER +if [[ $SKIP_DEPLOY != 'true' ]]; then + mvn -B -pl '!tools' -Dbuildver=301 -PsnapshotsWithDatabricks clean deploy $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER +else + mvn -B -pl '!tools' -Dbuildver=301 -PsnapshotsWithDatabricks clean install $MVN_URM_MIRROR -Dmaven.repo.local=$M2DIR -Dcuda.version=$CUDA_CLASSIFIER +fi # Parse cudf and spark files from local mvn repo jenkins/printJarVersion.sh "CUDFVersion" "$M2DIR/ai/rapids/cudf/${CUDF_VER}" "cudf-${CUDF_VER}" "-${CUDA_CLASSIFIER}.jar" $SERVER_ID From dd6b2b1c0840ce8450eed166d7d730964b12ad56 Mon Sep 17 00:00:00 2001 From: Peixin Date: Mon, 13 Dec 2021 16:43:16 +0800 Subject: [PATCH 49/52] Fix PYTHONPATH in pre-merge (#4349) * Fix PYTHONPATH in pre-merge Signed-off-by: Peixin Li * to the public location * explicitly add pythonpath for nightly script --- jenkins/spark-premerge-build.sh | 1 + jenkins/spark-tests.sh | 1 + 2 files changed, 2 insertions(+) diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 32df2c739fa..a82254ce240 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -141,6 +141,7 @@ export SPARK_HOME="$ARTF_ROOT/spark-$SPARK_VER-bin-hadoop3.2" export PATH="$SPARK_HOME/bin:$SPARK_HOME/sbin:$PATH" tar zxf $SPARK_HOME.tgz -C $ARTF_ROOT && \ rm -f $SPARK_HOME.tgz +export PYTHONPATH=$SPARK_HOME/python:$SPARK_HOME/python/pyspark/:$SPARK_HOME/python/lib/py4j-0.10.9-src.zip case $BUILD_TYPE in diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index be76e0978ba..8101d485b5d 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -106,6 +106,7 @@ export SPARK_HOME="$ARTF_ROOT/spark-$SPARK_VER-bin-hadoop3.2" export PATH="$SPARK_HOME/bin:$SPARK_HOME/sbin:$PATH" tar zxf $SPARK_HOME.tgz -C $ARTF_ROOT && \ rm -f $SPARK_HOME.tgz +export PYTHONPATH=$SPARK_HOME/python:$SPARK_HOME/python/pyspark/:$SPARK_HOME/python/lib/py4j-0.10.9-src.zip IS_SPARK_311_OR_LATER=0 [[ "$(printf '%s\n' "3.1.1" "$SPARK_VER" | sort -V | head -n1)" = "3.1.1" ]] && IS_SPARK_311_OR_LATER=1 From 49c36eabb6cbac3fdc6cb50f916bc1cd50c8c876 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 13 Dec 2021 08:45:54 -0700 Subject: [PATCH 50/52] Add support for regexp_extract on the GPU (#4285) * Implement regexp_extract Signed-off-by: Andy Grove * Add support for idx = 0 and add idx bounds checks Signed-off-by: Andy Grove * scalastyle and update docs Signed-off-by: Andy Grove * fix resource leak * rework error handling for idx out of range * move dataType definition up to GpuRegExpTernaryBase * address feedback * update no_match test to test other values for idx --- docs/compatibility.md | 4 +- docs/configs.md | 1 + docs/supported_ops.md | 627 +++++++++++------- .../src/main/python/string_test.py | 65 +- .../nvidia/spark/rapids/GpuOverrides.scala | 11 + .../spark/sql/rapids/stringFunctions.scala | 185 +++++- ...ite.scala => RegularExpressionSuite.scala} | 34 +- 7 files changed, 640 insertions(+), 287 deletions(-) rename tests/src/test/scala/com/nvidia/spark/rapids/{StringFallbackSuite.scala => RegularExpressionSuite.scala} (76%) diff --git a/docs/compatibility.md b/docs/compatibility.md index 7b4261f7df6..5b925aa36c3 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -446,6 +446,7 @@ The following Apache Spark regular expression functions and expressions are supp - `RLIKE` - `regexp` +- `regexp_extract` - `regexp_like` - `regexp_replace` @@ -457,6 +458,7 @@ These operations can be enabled on the GPU with the following configuration sett - `spark.rapids.sql.expression.RLike=true` (for `RLIKE`, `regexp`, and `regexp_like`) - `spark.rapids.sql.expression.RegExpReplace=true` for `regexp_replace` +- `spark.rapids.sql.expression.RegExpExtract=true` for `regexp_extract` Even when these expressions are enabled, there are instances where regular expression operations will fall back to CPU when the RAPIDS Accelerator determines that a pattern is either unsupported or would produce incorrect results on the GPU. @@ -475,8 +477,6 @@ Here are some examples of regular expression patterns that are not supported on In addition to these cases that can be detected, there are also known issues that can cause incorrect results: -- `$` does not match the end of a string if the string ends with a line-terminator - ([cuDF issue #9620](https://github.com/rapidsai/cudf/issues/9620)) - Character classes for negative matches have different behavior between CPU and GPU for multiline strings. The pattern `[^a]` will match line-terminators on CPU but not on GPU. diff --git a/docs/configs.md b/docs/configs.md index f0d6db7c4e2..796d8a612c9 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -262,6 +262,7 @@ Name | SQL Function(s) | Description | Default Value | Notes spark.rapids.sql.expression.RLike|`rlike`|RLike|false|This is disabled by default because the implementation is not 100% compatible. See the compatibility guide for more information.| spark.rapids.sql.expression.Rand|`random`, `rand`|Generate a random column with i.i.d. uniformly distributed values in [0, 1)|true|None| spark.rapids.sql.expression.Rank|`rank`|Window function that returns the rank value within the aggregation window|true|None| +spark.rapids.sql.expression.RegExpExtract|`regexp_extract`|RegExpExtract|false|This is disabled by default because the implementation is not 100% compatible. See the compatibility guide for more information.| spark.rapids.sql.expression.RegExpReplace|`regexp_replace`|RegExpReplace support for string literal input patterns|false|This is disabled by default because the implementation is not 100% compatible. See the compatibility guide for more information.| spark.rapids.sql.expression.Remainder|`%`, `mod`|Remainder or modulo|true|None| spark.rapids.sql.expression.Rint|`rint`|Rounds up a double value to the nearest double equal to an integer|true|None| diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 0597fe41d6f..9a46a116734 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -9823,6 +9823,95 @@ are limited. +RegExpExtract +`regexp_extract` +RegExpExtract +This is disabled by default because the implementation is not 100% compatible. See the compatibility guide for more information. +project +str + + + + + + + + + +S + + + + + + + + + + +regexp + + + + + + + + + +PS
Literal value only
+ + + + + + + + + + +idx + + + +S + + + + + + + + + + + + + + + + +result + + + + + + + + + +S + + + + + + + + + + RegExpReplace `regexp_replace` RegExpReplace support for string literal input patterns @@ -9980,6 +10069,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Rint `rint` Rounds up a double value to the nearest double equal to an integer @@ -10070,32 +10185,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Round `round` Round an expression to d decimal places using HALF_UP rounding mode @@ -10352,6 +10441,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + ShiftRight `shiftright` Bitwise shift right (>>) @@ -10488,32 +10603,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Signum `sign`, `signum` Returns -1.0, 0.0 or 1.0 as expr is negative, 0 or positive @@ -10741,6 +10830,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Size `size`, `cardinality` The size of an array or a map @@ -10856,32 +10971,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - SortOrder Sort order @@ -11113,6 +11202,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + StartsWith Starts with @@ -11270,32 +11385,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - StringLocate `position`, `locate` Substring search operator @@ -11474,6 +11563,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + StringRepeat `repeat` StringRepeat operator that repeats the given strings with numbers of times given by repeatTimes @@ -11607,54 +11722,28 @@ are limited. - - - -result - - - - - - - - - -S - - - - - - - - - - -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT + + + +result + + + + + + + + + +S + + + + + + + + StringSplit @@ -11882,6 +11971,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + StringTrimRight `rtrim` StringTrimRight operator @@ -12039,32 +12154,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - SubstringIndex `substring_index` substring_index operator @@ -12286,6 +12375,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Tan `tan` Tangent @@ -12466,32 +12581,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - TimeAdd Adds interval to timestamp @@ -12675,6 +12764,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + ToRadians `radians` Converts degrees to radians @@ -12858,32 +12973,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - TransformValues `transform_values` Transform values in a map using a transform function @@ -13042,6 +13131,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + UnaryPositive `positive` A numeric value with a + in front of it @@ -13252,32 +13367,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - UnscaledValue Convert a Decimal to an unscaled long value for some aggregation optimizations @@ -13419,6 +13508,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + WindowExpression Calculates a return value for every input row of a table based on a group (or "window") of rows diff --git a/integration_tests/src/main/python/string_test.py b/integration_tests/src/main/python/string_test.py index ffbad689653..7a7843b6357 100644 --- a/integration_tests/src/main/python/string_test.py +++ b/integration_tests/src/main/python/string_test.py @@ -14,7 +14,7 @@ import pytest -from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_are_equal_sql, assert_gpu_sql_fallback_collect, assert_gpu_fallback_collect +from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_are_equal_sql, assert_gpu_sql_fallback_collect, assert_gpu_fallback_collect, assert_gpu_and_cpu_error from conftest import is_databricks_runtime from data_gen import * from marks import * @@ -538,6 +538,69 @@ def test_regexp_replace_character_set_negated(): 'regexp_replace(a, "[^\n]", "1")'), conf={'spark.rapids.sql.expression.RegExpReplace': 'true'}) +def test_regexp_extract(): + gen = mk_str_gen('[abcd]{1,3}[0-9]{1,3}[abcd]{1,3}') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp_extract(a, "^([a-d]*)([0-9]*)([a-d]*)$", 1)', + 'regexp_extract(a, "^([a-d]*)([0-9]*)([a-d]*)$", 2)', + 'regexp_extract(a, "^([a-d]*)([0-9]*)([a-d]*)$", 3)'), + conf={'spark.rapids.sql.expression.RegExpExtract': 'true'}) + +def test_regexp_extract_no_match(): + gen = mk_str_gen('[abcd]{1,3}[0-9]{1,3}[abcd]{1,3}') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp_extract(a, "^([0-9]+)([a-z]+)([0-9]+)$", 0)', + 'regexp_extract(a, "^([0-9]+)([a-z]+)([0-9]+)$", 1)', + 'regexp_extract(a, "^([0-9]+)([a-z]+)([0-9]+)$", 2)', + 'regexp_extract(a, "^([0-9]+)([a-z]+)([0-9]+)$", 3)'), + conf={'spark.rapids.sql.expression.RegExpExtract': 'true'}) + +# if we determine that the index is out of range we fall back to CPU and let +# Spark take care of the error handling +@allow_non_gpu('ProjectExec', 'RegExpExtract') +def test_regexp_extract_idx_negative(): + gen = mk_str_gen('[abcd]{1,3}[0-9]{1,3}[abcd]{1,3}') + assert_gpu_and_cpu_error( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp_extract(a, "^([a-d]*)([0-9]*)([a-d]*)$", -1)').collect(), + error_message = "The specified group index cannot be less than zero", + conf={'spark.rapids.sql.expression.RegExpExtract': 'true'}) + +# if we determine that the index is out of range we fall back to CPU and let +# Spark take care of the error handling +@allow_non_gpu('ProjectExec', 'RegExpExtract') +def test_regexp_extract_idx_out_of_bounds(): + gen = mk_str_gen('[abcd]{1,3}[0-9]{1,3}[abcd]{1,3}') + assert_gpu_and_cpu_error( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp_extract(a, "^([a-d]*)([0-9]*)([a-d]*)$", 4)').collect(), + error_message = "Regex group count is 3, but the specified group index is 4", + conf={'spark.rapids.sql.expression.RegExpExtract': 'true'}) + +def test_regexp_extract_multiline(): + gen = mk_str_gen('[abcd]{2}[\r\n]{0,2}[0-9]{2}[\r\n]{0,2}[abcd]{2}') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp_extract(a, "^([a-d]*)([\r\n]*)", 2)'), + conf={'spark.rapids.sql.expression.RegExpExtract': 'true'}) + +def test_regexp_extract_multiline_negated_character_class(): + gen = mk_str_gen('[abcd]{2}[\r\n]{0,2}[0-9]{2}[\r\n]{0,2}[abcd]{2}') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp_extract(a, "^([a-d]*)([^a-z]*)([a-d]*)$", 2)'), + conf={'spark.rapids.sql.expression.RegExpExtract': 'true'}) + +def test_regexp_extract_idx_0(): + gen = mk_str_gen('[abcd]{1,3}[0-9]{1,3}[abcd]{1,3}') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'regexp_extract(a, "^([a-d]*)([0-9]*)([a-d]*)$", 0)', + 'regexp_extract(a, "^([a-d]*)[0-9]*([a-d]*)$", 0)'), + conf={'spark.rapids.sql.expression.RegExpExtract': 'true'}) + def test_rlike(): gen = mk_str_gen('[abcd]{1,3}') assert_gpu_and_cpu_are_equal_collect( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 58174ba1ccf..e40697b6b46 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3030,6 +3030,17 @@ object GpuOverrides extends Logging { (a, conf, p, r) => new GpuRLikeMeta(a, conf, p, r)).disabledByDefault( "the implementation is not 100% compatible. " + "See the compatibility guide for more information."), + expr[RegExpExtract]( + "RegExpExtract", + ExprChecks.projectOnly(TypeSig.STRING, TypeSig.STRING, + Seq(ParamCheck("str", TypeSig.STRING, TypeSig.STRING), + ParamCheck("regexp", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING), + ParamCheck("idx", TypeSig.lit(TypeEnum.INT), + TypeSig.lit(TypeEnum.INT)))), + (a, conf, p, r) => new GpuRegExpExtractMeta(a, conf, p, r)) + .disabledByDefault( + "the implementation is not 100% compatible. " + + "See the compatibility guide for more information."), expr[Length]( "String character length or binary byte length", ExprChecks.unaryProject(TypeSig.INT, TypeSig.INT, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index 6a14f74b561..692be261f6a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -23,7 +23,7 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.v2.ShimExpression -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ImplicitCastInputTypes, Literal, NullIntolerant, Predicate, RLike, StringSplit, SubstringIndex} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ImplicitCastInputTypes, Literal, NullIntolerant, Predicate, RegExpExtract, RLike, StringSplit, SubstringIndex} import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.UTF8String @@ -801,24 +801,9 @@ case class GpuRLike(left: Expression, right: Expression, pattern: String) override def dataType: DataType = BooleanType } -case class GpuRegExpReplace( - srcExpr: Expression, - searchExpr: Expression, - replaceExpr: Expression, - cudfRegexPattern: String) - extends GpuTernaryExpression with ImplicitCastInputTypes { +abstract class GpuRegExpTernaryBase extends GpuTernaryExpression { - override def dataType: DataType = srcExpr.dataType - - override def inputTypes: Seq[DataType] = Seq(StringType, StringType, StringType) - - override def first: Expression = srcExpr - override def second: Expression = searchExpr - override def third: Expression = replaceExpr - - def this(srcExpr: Expression, searchExpr: Expression, cudfRegexPattern: String) = { - this(srcExpr, searchExpr, GpuLiteral("", StringType), cudfRegexPattern) - } + override def dataType: DataType = StringType override def doColumnar( strExpr: GpuColumnVector, @@ -852,23 +837,171 @@ case class GpuRegExpReplace( override def doColumnar( strExpr: GpuColumnVector, - searchExpr: GpuScalar, - replaceExpr: GpuScalar): ColumnVector = { - strExpr.getBase.replaceRegex(cudfRegexPattern, replaceExpr.getBase) - } + searchExpr: GpuColumnVector, + replaceExpr: GpuScalar): ColumnVector = + throw new UnsupportedOperationException(s"Cannot columnar evaluate expression: $this") override def doColumnar(numRows: Int, val0: GpuScalar, val1: GpuScalar, val2: GpuScalar): ColumnVector = { - withResource(GpuColumnVector.from(val0, numRows, srcExpr.dataType)) { val0Col => + withResource(GpuColumnVector.from(val0, numRows, first.dataType)) { val0Col => doColumnar(val0Col, val1, val2) } } +} + +case class GpuRegExpReplace( + srcExpr: Expression, + searchExpr: Expression, + replaceExpr: Expression, + cudfRegexPattern: String) + extends GpuRegExpTernaryBase with ImplicitCastInputTypes { + + override def inputTypes: Seq[DataType] = Seq(StringType, StringType, StringType) + + override def first: Expression = srcExpr + override def second: Expression = searchExpr + override def third: Expression = replaceExpr + + def this(srcExpr: Expression, searchExpr: Expression, cudfRegexPattern: String) = { + this(srcExpr, searchExpr, GpuLiteral("", StringType), cudfRegexPattern) + } + override def doColumnar( strExpr: GpuColumnVector, - searchExpr: GpuColumnVector, - replaceExpr: GpuScalar): ColumnVector = - throw new UnsupportedOperationException(s"Cannot columnar evaluate expression: $this") + searchExpr: GpuScalar, + replaceExpr: GpuScalar): ColumnVector = { + strExpr.getBase.replaceRegex(cudfRegexPattern, replaceExpr.getBase) + } + +} + +class GpuRegExpExtractMeta( + expr: RegExpExtract, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends TernaryExprMeta[RegExpExtract](expr, conf, parent, rule) { + + private var pattern: Option[String] = None + private var numGroups = 0 + + override def tagExprForGpu(): Unit = { + + def countGroups(regexp: RegexAST): Int = { + regexp match { + case RegexGroup(_, term) => 1 + countGroups(term) + case other => other.children().map(countGroups).sum + } + } + + expr.regexp match { + case Literal(str: UTF8String, DataTypes.StringType) if str != null => + try { + val javaRegexpPattern = str.toString + // verify that we support this regex and can transpile it to cuDF format + val cudfRegexPattern = new CudfRegexTranspiler(replace = false) + .transpile(javaRegexpPattern) + pattern = Some(cudfRegexPattern) + numGroups = countGroups(new RegexParser(javaRegexpPattern).parse()) + } catch { + case e: RegexUnsupportedException => + willNotWorkOnGpu(e.getMessage) + } + case _ => + willNotWorkOnGpu(s"only non-null literal strings are supported on GPU") + } + + expr.idx match { + case Literal(value, DataTypes.IntegerType) => + val idx = value.asInstanceOf[Int] + if (idx < 0) { + willNotWorkOnGpu("the specified group index cannot be less than zero") + } + if (idx > numGroups) { + willNotWorkOnGpu( + s"regex group count is $numGroups, but the specified group index is $idx") + } + case _ => + willNotWorkOnGpu("GPU only supports literal index") + } + } + + override def convertToGpu( + str: Expression, + regexp: Expression, + idx: Expression): GpuExpression = { + val cudfPattern = pattern.getOrElse( + throw new IllegalStateException("Expression has not been tagged with cuDF regex pattern")) + GpuRegExpExtract(str, regexp, idx, cudfPattern) + } +} + +case class GpuRegExpExtract( + subject: Expression, + regexp: Expression, + idx: Expression, + cudfRegexPattern: String) + extends GpuRegExpTernaryBase with ImplicitCastInputTypes with NullIntolerant { + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) + override def first: Expression = subject + override def second: Expression = regexp + override def third: Expression = idx + + override def prettyName: String = "regexp_extract" + + override def doColumnar( + str: GpuColumnVector, + regexp: GpuScalar, + idx: GpuScalar): ColumnVector = { + + val groupIndex = idx.getValue.asInstanceOf[Int] + + // There are some differences in behavior between cuDF and Java so we have + // to handle those cases here. + // + // Given the pattern `^([a-z]*)([0-9]*)([a-z]*)$` the following table + // shows the value that would be extracted for group index 2 given a range + // of inputs. The behavior is mostly consistent except for the case where + // the input is non-null and does not match the pattern. + // + // | Input | Java | cuDF | + // |--------|-------|-------| + // | '' | '' | '' | + // | NULL | NULL | NULL | + // | 'a1a' | '1' | '1' | + // | '1a1' | '' | NULL | + + if (groupIndex == 0) { + withResource(GpuScalar.from("", DataTypes.StringType)) { emptyString => + withResource(GpuScalar.from(null, DataTypes.StringType)) { nullString => + withResource(str.getBase.matchesRe(cudfRegexPattern)) { matches => + withResource(str.getBase.isNull) { isNull => + withResource(matches.ifElse(str.getBase, emptyString)) { + isNull.ifElse(nullString, _) + } + } + } + } + } + } else { + withResource(GpuScalar.from("", DataTypes.StringType)) { emptyString => + withResource(GpuScalar.from(null, DataTypes.StringType)) { nullString => + withResource(str.getBase.extractRe(cudfRegexPattern)) { extract => + withResource(str.getBase.matchesRe(cudfRegexPattern)) { matches => + withResource(str.getBase.isNull) { isNull => + withResource(matches.ifElse(extract.getColumn(groupIndex - 1), emptyString)) { + isNull.ifElse(nullString, _) + } + } + } + } + } + } + } + } + } class SubstringIndexMeta( diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/StringFallbackSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionSuite.scala similarity index 76% rename from tests/src/test/scala/com/nvidia/spark/rapids/StringFallbackSuite.scala rename to tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionSuite.scala index 6ba3deed5c6..db3a1791317 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/StringFallbackSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionSuite.scala @@ -16,10 +16,13 @@ package com.nvidia.spark.rapids import org.apache.spark.SparkConf +import org.apache.spark.sql.{DataFrame, SparkSession} -class StringFallbackSuite extends SparkQueryCompareTestSuite { +class RegularExpressionSuite extends SparkQueryCompareTestSuite { - private val conf = new SparkConf().set("spark.rapids.sql.expression.RegExpReplace", "true") + private val conf = new SparkConf() + .set("spark.rapids.sql.expression.RegExpReplace", "true") + .set("spark.rapids.sql.expression.RegExpExtract", "true") testGpuFallback( "String regexp_replace replace str columnar fall back", @@ -90,4 +93,31 @@ class StringFallbackSuite extends SparkQueryCompareTestSuite { nullableStringsFromCsv, conf = conf) { frame => frame.selectExpr("regexp_replace(strings,'\\(foo\\)','D')") } + + testSparkResultsAreEqual("String regexp_extract regex 1", + extractStrings, conf = conf) { + frame => frame.selectExpr("regexp_extract(strings, '^([a-z]*)([0-9]*)([a-z]*)$', 1)") + } + + testSparkResultsAreEqual("String regexp_extract regex 2", + extractStrings, conf = conf) { + frame => frame.selectExpr("regexp_extract(strings, '^([a-z]*)([0-9]*)([a-z]*)$', 2)") + } + + testSparkResultsAreEqual("String regexp_extract literal input", + extractStrings, conf = conf) { + frame => frame.selectExpr("regexp_extract('abc123def', '^([a-z]*)([0-9]*)([a-z]*)$', 2)") + } + + private def extractStrings(session: SparkSession): DataFrame = { + import session.sqlContext.implicits._ + Seq[(String)]( + (""), + (null), + ("abc123def"), + ("abc\r\n12\r3\ndef"), + ("123abc456") + ).toDF("strings") + } + } From bc4cd096d25c90d468928b54fb91c934ec0b9649 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 14 Dec 2021 08:25:29 -0600 Subject: [PATCH 51/52] Fix issue with binding to hash agg columns with computation (#4343) Signed-off-by: Robert (Bobby) Evans --- .../src/main/python/hash_aggregate_test.py | 11 +++++++++++ .../scala/com/nvidia/spark/rapids/aggregate.scala | 15 +++++++++++---- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 9d236adcf71..516ba905617 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -329,6 +329,17 @@ def test_hash_reduction_sum_count_action(data_gen): lambda spark: gen_df(spark, data_gen, length=100).agg(f.sum('b')) ) +# Make sure that we can do computation in the group by columns +@ignore_order +def test_computation_in_grpby_columns(): + conf = {'spark.rapids.sql.batchSizeBytes' : '1000'} + data_gen = [ + ('a', RepeatSeqGen(StringGen('a{1,20}'), length=50)), + ('b', short_gen)] + assert_gpu_and_cpu_are_equal_collect( + lambda spark: gen_df(spark, data_gen).groupby(f.substring(f.col('a'), 2, 10)).agg(f.sum('b')), + conf = conf) + @shuffle_test @approximate_float @ignore_order diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index 58896717179..a90b3309163 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -423,8 +423,8 @@ class GpuHashAggregateIterator( } val shims = ShimLoader.getSparkShims - val ordering = groupingExpressions.map(shims.sortOrder(_, Ascending, NullsFirst)) val groupingAttributes = groupingExpressions.map(_.toAttribute) + val ordering = groupingAttributes.map(shims.sortOrder(_, Ascending, NullsFirst)) val aggBufferAttributes = groupingAttributes ++ aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) val sorter = new GpuSorter(ordering, aggBufferAttributes) @@ -644,8 +644,15 @@ class GpuHashAggregateIterator( private val postStep = new mutable.ArrayBuffer[Expression]() private val postStepAttr = new mutable.ArrayBuffer[Attribute]() - // we add the grouping expression first, which bind as pass-through - preStep ++= groupingExpressions + // we add the grouping expression first, which should bind as pass-through + if (forceMerge) { + // a grouping expression can do actual computation, but we cannot do that computation again + // on a merge, nor would we want to if we could. So use the attributes instead of the + // original expression when we are forcing a merge. + preStep ++= groupingAttributes + } else { + preStep ++= groupingExpressions + } postStep ++= groupingAttributes postStepAttr ++= groupingAttributes postStepDataTypes ++= @@ -679,7 +686,7 @@ class GpuHashAggregateIterator( // a bound expression that is applied before the cuDF aggregate private val preStepBound = if (forceMerge) { - GpuBindReferences.bindGpuReferences(preStep, aggBufferAttributes) + GpuBindReferences.bindGpuReferences(preStep.toList, aggBufferAttributes.toList) } else { GpuBindReferences.bindGpuReferences(preStep, inputAttributes) } From 3c8e5df9b8e82faab08d9ae3846ddbbdb0acf255 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 14 Dec 2021 14:14:23 -0700 Subject: [PATCH 52/52] Fix regression in AQE optimizations (#4354) * Fix regression in AQE Signed-off-by: Andy Grove * simplify test * sign-off Signed-off-by: Andy Grove --- .../spark/rapids/GpuTransitionOverrides.scala | 4 +-- .../spark/rapids/AdaptiveQueryExecSuite.scala | 25 +++++++++++++++++++ 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala index 566707b0dd9..e93bd20b223 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala @@ -76,14 +76,14 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { plan: SparkPlan, parent: Option[SparkPlan]): SparkPlan = plan match { - case GpuBringBackToHost(child) if parent.isEmpty => + case bb @ GpuBringBackToHost(child) if parent.isEmpty => // This is hacky but we need to remove the GpuBringBackToHost from the final // query stage, if there is one. It gets inserted by // GpuTransitionOverrides.insertColumnarFromGpu around columnar adaptive // plans when we are writing to columnar formats on the GPU. It would be nice to avoid // inserting it in the first place but we just don't have enough context // at the time GpuTransitionOverrides is applying rules. - child + optimizeAdaptiveTransitions(child, Some(bb)) // HostColumnarToGpu(RowToColumnarExec(..)) => GpuRowToColumnarExec(..) case HostColumnarToGpu(r2c: RowToColumnarExec, goal) => diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala index bb494f9e9d1..2710cdcda7a 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala @@ -386,6 +386,31 @@ class AdaptiveQueryExecSuite }, conf) } + // repro case for https://github.com/NVIDIA/spark-rapids/issues/4351 + test("Write parquet from AQE shuffle with limit") { + logError("Write parquet from AQE shuffle with limit") + + val conf = new SparkConf() + .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + + withGpuSparkSession(spark => { + import spark.implicits._ + + val path = new File(TEST_FILES_ROOT, "AvoidTransitionInput.parquet").getAbsolutePath + (0 until 100).toDF("a") + .write + .mode(SaveMode.Overwrite) + .parquet(path) + + val outputPath = new File(TEST_FILES_ROOT, "AvoidTransitionOutput.parquet").getAbsolutePath + spark.read.parquet(path) + .limit(100) + .write.mode(SaveMode.Overwrite) + .parquet(outputPath) + }, conf) + } + + test("Exchange reuse") { logError("Exchange reuse") assumeSpark301orLater