From b67d52b0a770e8cd284b31ed83b02633d8576672 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 13 Mar 2024 18:32:51 +0800 Subject: [PATCH 01/11] Use new kernel for getJsonObject Signed-off-by: Haoyang Li --- .../spark/rapids/GpuGetJsonObject.scala | 76 ++++++------------- .../nvidia/spark/rapids/GpuOverrides.scala | 7 +- 2 files changed, 29 insertions(+), 54 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala index 16950368ab0..f15b3c07e4c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala @@ -18,12 +18,12 @@ package com.nvidia.spark.rapids import scala.util.parsing.combinator.RegexParsers -import ai.rapids.cudf.{ColumnVector, GetJsonObjectOptions, Scalar} +import ai.rapids.cudf.ColumnVector import com.nvidia.spark.rapids.Arm.withResource +import com.nvidia.spark.rapids.jni.JSONUtils -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, GetJsonObject} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression} import org.apache.spark.sql.types.{DataType, StringType} -import org.apache.spark.unsafe.types.UTF8String // Copied from Apache Spark org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala private[this] sealed trait PathInstruction @@ -83,49 +83,22 @@ private[this] object JsonPathParser extends RegexParsers { } } - def containsUnsupportedPath(instructions: List[PathInstruction]): Boolean = { - // Gpu GetJsonObject is not supported if JSON path contains wildcard [*] - // see https://github.com/NVIDIA/spark-rapids/issues/10216 - instructions.exists { - case Wildcard => true - case Named(name) if name == "*" => true - case _ => false + def unzipInstruction(instruction: PathInstruction): (Int, String, Long) = { + instruction match { + case Subscript => (0, "", -1) + case Wildcard => (1, "", -1) + case Key => (2, "", -1) + case Index(index) => (3, "", index) + case Named(name) => (4, name, -1) } } - def normalize(instructions: List[PathInstruction]): String = { - // convert List[PathInstruction] to String - "$" + instructions.map { - case Subscript | Key => "" - case Wildcard => "[*]" - case Index(index) => s"[$index]" - case Named(name) => s"['$name']" - case _ => throw new IllegalArgumentException(s"Invalid instruction in path") - }.mkString + def splitInstructions(instructions: List[PathInstruction]): + (List[Int], List[String], List[Long]) = { + instructions.map(unzipInstruction).unzip3 } } -class GpuGetJsonObjectMeta( - expr: GetJsonObject, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: DataFromReplacementRule - ) extends BinaryExprMeta[GetJsonObject](expr, conf, parent, rule) { - - override def tagExprForGpu(): Unit = { - val lit = GpuOverrides.extractLit(expr.right) - lit.map { l => - val instructions = JsonPathParser.parse(l.value.asInstanceOf[UTF8String].toString) - if (instructions.exists(JsonPathParser.containsUnsupportedPath)) { - willNotWorkOnGpu("get_json_object on GPU does not support wildcard [*] in path") - } - } - } - - override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = - GpuGetJsonObject(lhs, rhs) -} - case class GpuGetJsonObject(json: Expression, path: Expression) extends GpuBinaryExpressionArgsAnyScalar with ExpectsInputTypes { @@ -136,28 +109,29 @@ case class GpuGetJsonObject(json: Expression, path: Expression) override def nullable: Boolean = true override def prettyName: String = "get_json_object" - private var cachedNormalizedPath: Option[Option[String]] = None + private var cachedInstructions: + Option[Option[(List[Int], List[String], List[Long])]] = None - def normalizeJsonPath(path: GpuScalar): Option[String] = { + def normalizeJsonPath(path: GpuScalar): Option[(List[Int], List[String], List[Long])] = { if (path.isValid) { val pathStr = path.getValue.toString() - JsonPathParser.parse(pathStr).map(JsonPathParser.normalize) + JsonPathParser.parse(pathStr).map(JsonPathParser.splitInstructions) } else { None } } override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { - cachedNormalizedPath.getOrElse { - val normalizedPath: Option[String] = normalizeJsonPath(rhs) - cachedNormalizedPath = Some(normalizedPath) - normalizedPath + cachedInstructions.getOrElse { + val pathInstructions = normalizeJsonPath(rhs) + cachedInstructions = Some(pathInstructions) + pathInstructions } match { - case Some(normalizedStr) => - withResource(Scalar.fromString(normalizedStr)) { scalar => - lhs.getBase().getJSONObject(scalar, - GetJsonObjectOptions.builder().allowSingleQuotes(true).build()) + case Some(instructions) => instructions match { + case (a: List[Int], b: List[String], c: List[Long]) => { + JSONUtils.getJsonObject(lhs.getBase, a.toArray, b.toArray, c.toArray) } + } case None => GpuColumnVector.columnVectorFromNull(lhs.getRowCount.toInt, StringType) } } 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 38562dfdb2f..4264233dea4 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 @@ -3651,9 +3651,10 @@ object GpuOverrides extends Logging { ExprChecks.projectOnly( TypeSig.STRING, TypeSig.STRING, Seq(ParamCheck("json", TypeSig.STRING, TypeSig.STRING), ParamCheck("path", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING))), - (a, conf, p, r) => new GpuGetJsonObjectMeta(a, conf, p, r) - ).disabledByDefault("escape sequences are not processed correctly, the input is not " + - "validated, and the output is not normalized the same as Spark"), + (a, conf, p, r) => new BinaryExprMeta[GetJsonObject](a, conf, p, r) { + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = + GpuGetJsonObject(lhs, rhs) + }), expr[JsonToStructs]( "Returns a struct value with the given `jsonStr` and `schema`", ExprChecks.projectOnly( From a5ea88b2b1114679673eb90d49b6ead1521a77e8 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Fri, 15 Mar 2024 13:06:02 +0800 Subject: [PATCH 02/11] Use table to pass parsed path Signed-off-by: Haoyang Li --- .../advanced_configs.md | 2 +- docs/supported_ops.md | 2 +- .../src/main/python/get_json_test.py | 32 ++++-------- .../spark/rapids/GpuGetJsonObject.scala | 51 ++++++++++++------- tools/generated_files/311/supportedExprs.csv | 6 +-- tools/generated_files/341/supportedExprs.csv | 6 +-- tools/generated_files/supportedExprs.csv | 6 +-- 7 files changed, 55 insertions(+), 50 deletions(-) diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index dacf6cf4667..546d43e3b6b 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -248,7 +248,7 @@ Name | SQL Function(s) | Description | Default Value | Notes spark.rapids.sql.expression.FromUnixTime|`from_unixtime`|Get the string from a unix timestamp|true|None| spark.rapids.sql.expression.GetArrayItem| |Gets the field at `ordinal` in the Array|true|None| spark.rapids.sql.expression.GetArrayStructFields| |Extracts the `ordinal`-th fields of all array elements for the data with the type of array of struct|true|None| -spark.rapids.sql.expression.GetJsonObject|`get_json_object`|Extracts a json object from path|false|This is disabled by default because escape sequences are not processed correctly, the input is not validated, and the output is not normalized the same as Spark| +spark.rapids.sql.expression.GetJsonObject|`get_json_object`|Extracts a json object from path|true|None| spark.rapids.sql.expression.GetMapValue| |Gets Value from a Map based on a key|true|None| spark.rapids.sql.expression.GetStructField| |Gets the named field of the struct|true|None| spark.rapids.sql.expression.GetTimestamp| |Gets timestamps from strings using given pattern.|true|None| diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 65873a40a5a..2ebcd9ed423 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -6856,7 +6856,7 @@ are limited. GetJsonObject `get_json_object` Extracts a json object from path -This is disabled by default because escape sequences are not processed correctly, the input is not validated, and the output is not normalized the same as Spark +None project json diff --git a/integration_tests/src/main/python/get_json_test.py b/integration_tests/src/main/python/get_json_test.py index 935a61e0562..8cbe869835e 100644 --- a/integration_tests/src/main/python/get_json_test.py +++ b/integration_tests/src/main/python/get_json_test.py @@ -85,23 +85,8 @@ def test_get_json_object_single_quotes(): pytest.param("$.a",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10196')), "$.non_exist_key", "$..no_recursive", - "$.store.book[0].non_exist_key"]) -def test_get_json_object_spark_unit_tests(query): - schema = StructType([StructField("jsonStr", StringType())]) - data = [ - ['''{"store":{"fruit":[{"weight":8,"type":"apple"},{"weight":9,"type":"pear"}],"basket":[[1,2,{"b":"y","a":"x"}],[3,4],[5,6]],"book":[{"author":"Nigel Rees","title":"Sayings of the Century","category":"reference","price":8.95},{"author":"Herman Melville","title":"Moby Dick","category":"fiction","price":8.99,"isbn":"0-553-21311-3"},{"author":"J. R. R. Tolkien","title":"The Lord of the Rings","category":"fiction","reader":[{"age":25,"name":"bob"},{"age":26,"name":"jack"}],"price":22.99,"isbn":"0-395-19395-8"}],"bicycle":{"price":19.95,"color":"red"}},"email":"amy@only_for_json_udf_test.net","owner":"amy","zip code":"94025","fb:testid":"1234"}'''], - ['''{ "key with spaces": "it works" }'''], - ['''{"a":"b\nc"}'''], - ['''{"a":"b\"c"}'''], - ["\u0000\u0000\u0000A\u0001AAA"], - ['{"big": "' + ('x' * 3000) + '"}']] - assert_gpu_and_cpu_are_equal_collect( - lambda spark: spark.createDataFrame(data,schema=schema).select( - f.get_json_object('jsonStr', query)), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) - -@allow_non_gpu("ProjectExec", "GetJsonObject") -@pytest.mark.parametrize('query',["$.store.basket[0][*].b", + "$.store.book[0].non_exist_key", + "$.store.basket[0][*].b", "$.store.book[*].reader", "$.store.book[*]", "$.store.book[*].category", @@ -111,13 +96,18 @@ def test_get_json_object_spark_unit_tests(query): "$.store.basket[0][*]", "$.store.basket[*][*]", "$.store.basket[*].non_exist_key"]) -def test_get_json_object_spark_unit_tests_fallback(query): +def test_get_json_object_spark_unit_tests(query): schema = StructType([StructField("jsonStr", StringType())]) - data = [['''{"store":{"fruit":[{"weight":8,"type":"apple"},{"weight":9,"type":"pear"}],"basket":[[1,2,{"b":"y","a":"x"}],[3,4],[5,6]],"book":[{"author":"Nigel Rees","title":"Sayings of the Century","category":"reference","price":8.95},{"author":"Herman Melville","title":"Moby Dick","category":"fiction","price":8.99,"isbn":"0-553-21311-3"},{"author":"J. R. R. Tolkien","title":"The Lord of the Rings","category":"fiction","reader":[{"age":25,"name":"bob"},{"age":26,"name":"jack"}],"price":22.99,"isbn":"0-395-19395-8"}],"bicycle":{"price":19.95,"color":"red"}},"email":"amy@only_for_json_udf_test.net","owner":"amy","zip code":"94025","fb:testid":"1234"}''']] - assert_gpu_fallback_collect( + data = [ + ['''{"store":{"fruit":[{"weight":8,"type":"apple"},{"weight":9,"type":"pear"}],"basket":[[1,2,{"b":"y","a":"x"}],[3,4],[5,6]],"book":[{"author":"Nigel Rees","title":"Sayings of the Century","category":"reference","price":8.95},{"author":"Herman Melville","title":"Moby Dick","category":"fiction","price":8.99,"isbn":"0-553-21311-3"},{"author":"J. R. R. Tolkien","title":"The Lord of the Rings","category":"fiction","reader":[{"age":25,"name":"bob"},{"age":26,"name":"jack"}],"price":22.99,"isbn":"0-395-19395-8"}],"bicycle":{"price":19.95,"color":"red"}},"email":"amy@only_for_json_udf_test.net","owner":"amy","zip code":"94025","fb:testid":"1234"}'''], + ['''{ "key with spaces": "it works" }'''], + ['''{"a":"b\nc"}'''], + ['''{"a":"b\"c"}'''], + ["\u0000\u0000\u0000A\u0001AAA"], + ['{"big": "' + ('x' * 3000) + '"}']] + assert_gpu_and_cpu_are_equal_collect( lambda spark: spark.createDataFrame(data,schema=schema).select( f.get_json_object('jsonStr', query)), - "GetJsonObject", conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) @pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/10218") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala index f15b3c07e4c..07940dd28fd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala @@ -18,6 +18,7 @@ package com.nvidia.spark.rapids import scala.util.parsing.combinator.RegexParsers +import ai.rapids.cudf import ai.rapids.cudf.ColumnVector import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.jni.JSONUtils @@ -26,8 +27,8 @@ import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression} import org.apache.spark.sql.types.{DataType, StringType} // Copied from Apache Spark org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala -private[this] sealed trait PathInstruction -private[this] object PathInstruction { +sealed trait PathInstruction +object PathInstruction { case object Subscript extends PathInstruction case object Wildcard extends PathInstruction case object Key extends PathInstruction @@ -35,7 +36,7 @@ private[this] object PathInstruction { case class Named(name: String) extends PathInstruction } -private[this] object JsonPathParser extends RegexParsers { +object JsonPathParser extends RegexParsers { import PathInstruction._ def root: Parser[Char] = '$' @@ -83,19 +84,33 @@ private[this] object JsonPathParser extends RegexParsers { } } - def unzipInstruction(instruction: PathInstruction): (Int, String, Long) = { + def unzipInstruction(instruction: PathInstruction): (String, String, Long) = { instruction match { - case Subscript => (0, "", -1) - case Wildcard => (1, "", -1) - case Key => (2, "", -1) - case Index(index) => (3, "", index) - case Named(name) => (4, name, -1) + case Subscript => ("subscript", "", -1) + case Key => ("key", "", -1) + case Wildcard => ("wildcard", "", -1) + case Index(index) => ("index", "", index) + case Named(name) => ("named", name, -1) } } def splitInstructions(instructions: List[PathInstruction]): - (List[Int], List[String], List[Long]) = { - instructions.map(unzipInstruction).unzip3 + (Array[String], Array[String], Array[Long]) = { + instructions.map(unzipInstruction).unzip3 match { + case (types, names, values) => + (types.toArray, names.toArray, values.toArray) + } + } + + def createTable(instructions: List[PathInstruction]): cudf.Table = { + val (types, names, values) = splitInstructions(instructions) + withResource(ColumnVector.fromStrings(types: _*)) { typesColumn => + withResource(ColumnVector.fromStrings(names: _*)) { namesColumn => + withResource(ColumnVector.fromLongs(values: _*)) { valuesColumn => + new cudf.Table(typesColumn, namesColumn, valuesColumn) + } + } + } } } @@ -110,12 +125,12 @@ case class GpuGetJsonObject(json: Expression, path: Expression) override def prettyName: String = "get_json_object" private var cachedInstructions: - Option[Option[(List[Int], List[String], List[Long])]] = None + Option[Option[List[PathInstruction]]] = None - def normalizeJsonPath(path: GpuScalar): Option[(List[Int], List[String], List[Long])] = { + def parseJsonPath(path: GpuScalar): Option[List[PathInstruction]] = { if (path.isValid) { val pathStr = path.getValue.toString() - JsonPathParser.parse(pathStr).map(JsonPathParser.splitInstructions) + JsonPathParser.parse(pathStr) } else { None } @@ -123,13 +138,13 @@ case class GpuGetJsonObject(json: Expression, path: Expression) override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { cachedInstructions.getOrElse { - val pathInstructions = normalizeJsonPath(rhs) + val pathInstructions = parseJsonPath(rhs) cachedInstructions = Some(pathInstructions) pathInstructions } match { - case Some(instructions) => instructions match { - case (a: List[Int], b: List[String], c: List[Long]) => { - JSONUtils.getJsonObject(lhs.getBase, a.toArray, b.toArray, c.toArray) + case Some(instructions) => { + withResource(JsonPathParser.createTable(instructions)) { instructions => + JSONUtils.getJsonObject(lhs.getBase, instructions) } } case None => GpuColumnVector.columnVectorFromNull(lhs.getRowCount.toInt, StringType) diff --git a/tools/generated_files/311/supportedExprs.csv b/tools/generated_files/311/supportedExprs.csv index 52cd9957729..b3b74b9fd5e 100644 --- a/tools/generated_files/311/supportedExprs.csv +++ b/tools/generated_files/311/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS diff --git a/tools/generated_files/341/supportedExprs.csv b/tools/generated_files/341/supportedExprs.csv index 519fa951e3e..a6e8c8269c5 100644 --- a/tools/generated_files/341/supportedExprs.csv +++ b/tools/generated_files/341/supportedExprs.csv @@ -235,9 +235,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv index 52cd9957729..b3b74b9fd5e 100644 --- a/tools/generated_files/supportedExprs.csv +++ b/tools/generated_files/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS From 257a6d6aab14bc7068e876fbca385a5ced48c785 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Thu, 21 Mar 2024 09:33:13 +0800 Subject: [PATCH 03/11] use list/vector of instruction objects Signed-off-by: Haoyang Li --- .../src/main/python/get_json_test.py | 4 +-- .../spark/rapids/GpuGetJsonObject.scala | 36 ++++++++----------- 2 files changed, 16 insertions(+), 24 deletions(-) diff --git a/integration_tests/src/main/python/get_json_test.py b/integration_tests/src/main/python/get_json_test.py index 8cbe869835e..6cf3dbd8b3e 100644 --- a/integration_tests/src/main/python/get_json_test.py +++ b/integration_tests/src/main/python/get_json_test.py @@ -55,9 +55,7 @@ def test_get_json_object_quoted_index(): DB 10.4 shows incorrect behaviour with single quotes") def test_get_json_object_single_quotes(): schema = StructType([StructField("jsonStr", StringType())]) - data = [[r'''{'a':'A'}'''], - [r'''{'b':'"B'}'''], - [r'''{"c":"'C"}''']] + data = [[r'''{'a':'A'}''']] assert_gpu_and_cpu_are_equal_collect( lambda spark: spark.createDataFrame(data,schema=schema).select( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala index 07940dd28fd..bab0dde5e7f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala @@ -18,7 +18,6 @@ package com.nvidia.spark.rapids import scala.util.parsing.combinator.RegexParsers -import ai.rapids.cudf import ai.rapids.cudf.ColumnVector import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.jni.JSONUtils @@ -94,23 +93,18 @@ object JsonPathParser extends RegexParsers { } } - def splitInstructions(instructions: List[PathInstruction]): - (Array[String], Array[String], Array[Long]) = { - instructions.map(unzipInstruction).unzip3 match { - case (types, names, values) => - (types.toArray, names.toArray, values.toArray) - } - } - - def createTable(instructions: List[PathInstruction]): cudf.Table = { - val (types, names, values) = splitInstructions(instructions) - withResource(ColumnVector.fromStrings(types: _*)) { typesColumn => - withResource(ColumnVector.fromStrings(names: _*)) { namesColumn => - withResource(ColumnVector.fromLongs(values: _*)) { valuesColumn => - new cudf.Table(typesColumn, namesColumn, valuesColumn) - } - } - } + def convertToJniObject(instructions: List[PathInstruction]): + Array[JSONUtils.PathInstructionJni] = { + instructions.map { instruction => + val (tpe, name, index) = unzipInstruction(instruction) + new JSONUtils.PathInstructionJni(tpe match { + case "subscript" => JSONUtils.PathInstructionType.SUBSCRIPT + case "key" => JSONUtils.PathInstructionType.KEY + case "wildcard" => JSONUtils.PathInstructionType.WILDCARD + case "index" => JSONUtils.PathInstructionType.INDEX + case "named" => JSONUtils.PathInstructionType.NAMED + }, name, index) + }.toArray } } @@ -143,9 +137,9 @@ case class GpuGetJsonObject(json: Expression, path: Expression) pathInstructions } match { case Some(instructions) => { - withResource(JsonPathParser.createTable(instructions)) { instructions => - JSONUtils.getJsonObject(lhs.getBase, instructions) - } + val jniInstructions = JsonPathParser.convertToJniObject(instructions) + val insSize = jniInstructions.length + JSONUtils.getJsonObject(lhs.getBase, insSize, jniInstructions) } case None => GpuColumnVector.columnVectorFromNull(lhs.getRowCount.toInt, StringType) } From baad1e589b3bf47abefec3c44018147527760ecc Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Sun, 24 Mar 2024 17:04:26 +0800 Subject: [PATCH 04/11] fallback when nested too long Signed-off-by: Haoyang Li --- .../src/main/python/get_json_test.py | 18 +++------- .../src/main/python/json_matrix_test.py | 34 ++++++------------- .../spark/rapids/GpuGetJsonObject.scala | 26 ++++++++++++++ .../nvidia/spark/rapids/GpuOverrides.scala | 5 +-- 4 files changed, 42 insertions(+), 41 deletions(-) diff --git a/integration_tests/src/main/python/get_json_test.py b/integration_tests/src/main/python/get_json_test.py index 6cf3dbd8b3e..ad0ab9069f9 100644 --- a/integration_tests/src/main/python/get_json_test.py +++ b/integration_tests/src/main/python/get_json_test.py @@ -37,8 +37,7 @@ def test_get_json_object(json_str_pattern): 'get_json_object(a, "$.store.fruit[0]")', 'get_json_object(\'%s\', "$.store.fruit[0]")' % scalar_json, ), - conf={'spark.sql.parser.escapedStringLiterals': 'true', - 'spark.rapids.sql.expression.GetJsonObject': 'true'}) + conf={'spark.sql.parser.escapedStringLiterals': 'true'}) def test_get_json_object_quoted_index(): schema = StructType([StructField("jsonStr", StringType())]) @@ -48,8 +47,7 @@ def test_get_json_object_quoted_index(): assert_gpu_and_cpu_are_equal_collect( lambda spark: spark.createDataFrame(data,schema=schema).select( f.get_json_object('jsonStr',r'''$['a']''').alias('sub_a'), - f.get_json_object('jsonStr',r'''$['b']''').alias('sub_b')), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + f.get_json_object('jsonStr',r'''$['b']''').alias('sub_b'))) @pytest.mark.skipif(is_databricks_runtime() and not is_databricks113_or_later(), reason="get_json_object on \ DB 10.4 shows incorrect behaviour with single quotes") @@ -72,15 +70,13 @@ def test_get_json_object_single_quotes(): "$.store.book", "$.store.book[0]", pytest.param("$",marks=[ - pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218'), - pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10196'), - pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10194')]), + pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')]), "$.store.book[0].category", "$.store.basket[0][1]", "$.store.basket[0][2].b", "$.zip code", "$.fb:testid", - pytest.param("$.a",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10196')), + "$.a", "$.non_exist_key", "$..no_recursive", "$.store.book[0].non_exist_key", @@ -139,7 +135,6 @@ def test_get_json_object_quoted_question(): f.get_json_object('jsonStr',r'''$['?']''').alias('question')), conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) -@pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/10196") def test_get_json_object_escaped_string_data(): schema = StructType([StructField("jsonStr", StringType())]) data = [[r'{"a":"A\"B"}'], @@ -155,7 +150,6 @@ def test_get_json_object_escaped_string_data(): lambda spark: spark.createDataFrame(data,schema=schema).selectExpr('get_json_object(jsonStr,"$.a")'), conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) -@pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/10196") def test_get_json_object_escaped_key(): schema = StructType([StructField("jsonStr", StringType())]) data = [ @@ -194,7 +188,6 @@ def test_get_json_object_escaped_key(): ), conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) -@pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/10212") def test_get_json_object_invalid_path(): schema = StructType([StructField("jsonStr", StringType())]) data = [['{"a":"A"}'], @@ -303,8 +296,7 @@ def assert_gpu_did_fallback(sql_text): assert_gpu_fallback_collect(lambda spark: gen_df(spark, [('a', gen), ('b', pattern)], length=10).selectExpr(sql_text), 'GetJsonObject', - conf={'spark.sql.parser.escapedStringLiterals': 'true', - 'spark.rapids.sql.expression.GetJsonObject': 'true'}) + conf={'spark.sql.parser.escapedStringLiterals': 'true'}) assert_gpu_did_fallback('get_json_object(a, b)') assert_gpu_did_fallback('get_json_object(\'%s\', b)' % scalar_json) diff --git a/integration_tests/src/main/python/json_matrix_test.py b/integration_tests/src/main/python/json_matrix_test.py index 823d8e0fe0e..f36e5c7477d 100644 --- a/integration_tests/src/main/python/json_matrix_test.py +++ b/integration_tests/src/main/python/json_matrix_test.py @@ -62,10 +62,6 @@ def read_json_as_text(spark, data_path, column_name): 'spark.rapids.sql.json.read.decimal.enabled': 'true' } -_enable_get_json_object_conf = { - 'spark.rapids.sql.expression.GetJsonObject': 'true' -} - _enable_json_tuple_conf = { 'spark.rapids.sql.expression.JsonTuple': 'true' } @@ -115,8 +111,7 @@ def test_from_json_allow_comments_off(std_input_path): @pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10194') def test_get_json_object_allow_comments_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( - lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_COMMENTS_FILE, "json").selectExpr('''get_json_object(json, "$.str")'''), - conf =_enable_get_json_object_conf) + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_COMMENTS_FILE, "json").selectExpr('''get_json_object(json, "$.str")''')) # Off is the default so it really needs to work @allow_non_gpu(TEXT_INPUT_EXEC) @@ -170,8 +165,7 @@ def test_from_json_allow_single_quotes_on(std_input_path): @allow_non_gpu(TEXT_INPUT_EXEC) def test_get_json_object_allow_single_quotes_on(std_input_path): assert_gpu_and_cpu_are_equal_collect( - lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_SQ_FILE, "json").selectExpr('''get_json_object(json, "$.str")'''), - conf =_enable_get_json_object_conf) + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_SQ_FILE, "json").selectExpr('''get_json_object(json, "$.str")''')) # On is the default so it really needs to work @allow_non_gpu(TEXT_INPUT_EXEC) @@ -225,8 +219,7 @@ def test_from_json_allow_unquoted_field_names_on(std_input_path): @pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10454') def test_get_json_object_allow_unquoted_field_names_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( - lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTE_FIELD_NAMES_FILE, "json").selectExpr('''get_json_object(json, "$.str")'''), - conf =_enable_get_json_object_conf) + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTE_FIELD_NAMES_FILE, "json").selectExpr('''get_json_object(json, "$.str")''')) # Off is the default so it really needs to work @allow_non_gpu(TEXT_INPUT_EXEC) @@ -287,8 +280,7 @@ def test_from_json_allow_numeric_leading_zeros_off(std_input_path): def test_get_json_object_allow_numeric_leading_zeros_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NUMERIC_LEAD_ZEROS_FILE, "json").selectExpr('''get_json_object(json, "$.byte")''', - '''get_json_object(json, "$.int")''', '''get_json_object(json, "$.float")''','''get_json_object(json, "$.decimal")'''), - conf =_enable_get_json_object_conf) + '''get_json_object(json, "$.int")''', '''get_json_object(json, "$.float")''','''get_json_object(json, "$.decimal")''')) # Off is the default so it really needs to work @allow_non_gpu(TEXT_INPUT_EXEC) @@ -351,8 +343,7 @@ def test_from_json_allow_nonnumeric_numbers_on(std_input_path): def test_get_json_object_allow_nonnumeric_numbers_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NONNUMERIC_NUMBERS_FILE, "json").selectExpr('''get_json_object(json, "$.float")''', - '''get_json_object(json, "$.double")'''), - conf =_enable_get_json_object_conf) + '''get_json_object(json, "$.double")''')) # Off is the default for json_tuple, so we want this to work @allow_non_gpu(TEXT_INPUT_EXEC) @@ -408,8 +399,7 @@ def test_from_json_allow_backslash_escape_any_on(std_input_path): @pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10454') def test_get_json_object_allow_backslash_escape_any_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( - lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_BS_ESC_FILE, "json").selectExpr('''get_json_object(json, "$.str")'''), - conf =_enable_get_json_object_conf) + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_BS_ESC_FILE, "json").selectExpr('''get_json_object(json, "$.str")''')) # Off is the default for json_tuple, so we want this to work @allow_non_gpu(TEXT_INPUT_EXEC) @@ -463,8 +453,7 @@ def test_from_json_allow_unquoted_control_chars_on(std_input_path): @allow_non_gpu(TEXT_INPUT_EXEC) def test_get_json_object_allow_unquoted_control_chars_on(std_input_path): assert_gpu_and_cpu_are_equal_collect( - lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTED_CONTROL_FILE, "json").selectExpr('''get_json_object(json, "$.str")'''), - conf =_enable_get_json_object_conf) + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTED_CONTROL_FILE, "json").selectExpr('''get_json_object(json, "$.str")''')) # On is the default for json_tuple, so we want this to work @allow_non_gpu(TEXT_INPUT_EXEC) @@ -522,8 +511,7 @@ def test_from_json_dec_locale(std_input_path, locale): @allow_non_gpu(TEXT_INPUT_EXEC) def test_get_json_object_dec_locale(std_input_path): assert_gpu_and_cpu_are_equal_collect( - lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_FILE, "json").selectExpr('''get_json_object(json, "$.data")'''), - conf =_enable_get_json_object_conf) + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_FILE, "json").selectExpr('''get_json_object(json, "$.data")''')) #There is no way to set a locale for these, and it really should not matter @allow_non_gpu(TEXT_INPUT_EXEC) @@ -581,8 +569,7 @@ def test_from_json_dec_locale_non_aribic(std_input_path, locale): @allow_non_gpu(TEXT_INPUT_EXEC) def test_get_json_object_dec_locale_non_aribic(std_input_path): assert_gpu_and_cpu_are_equal_collect( - lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_NON_ARIBIC_FILE, "json").selectExpr('''get_json_object(json, "$.data")'''), - conf =_enable_get_json_object_conf) + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_NON_ARIBIC_FILE, "json").selectExpr('''get_json_object(json, "$.data")''')) #There is no way to set a locale for these, and it really should not matter @allow_non_gpu(TEXT_INPUT_EXEC) @@ -742,8 +729,7 @@ def test_from_json_strings(std_input_path, input_file): @allow_non_gpu(TEXT_INPUT_EXEC) def test_get_json_object_formats(std_input_path, input_file): assert_gpu_and_cpu_are_equal_collect( - lambda spark : read_json_as_text(spark, std_input_path + '/' + input_file, "json").selectExpr("*", '''get_json_object(json, "$.data")'''), - conf =_enable_get_json_object_conf) + lambda spark : read_json_as_text(spark, std_input_path + '/' + input_file, "json").selectExpr("*", '''get_json_object(json, "$.data")''')) @pytest.mark.parametrize('input_file', [ pytest.param("int_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala index bab0dde5e7f..86e3a9b7d85 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala @@ -83,6 +83,11 @@ object JsonPathParser extends RegexParsers { } } + def fallbackCheck(instructions: List[PathInstruction]): Boolean = { + // JNI kernel has a limit of 16 nested nodes, fallback to CPU if we exceed that + instructions.filterNot(_ == Subscript | Key).length > 16 + } + def unzipInstruction(instruction: PathInstruction): (String, String, Long) = { instruction match { case Subscript => ("subscript", "", -1) @@ -108,6 +113,27 @@ object JsonPathParser extends RegexParsers { } } +class GpuGetJsonObjectMeta( + expr: GetJsonObject, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule + ) extends BinaryExprMeta[GetJsonObject](expr, conf, parent, rule) { + + override def tagExprForGpu(): Unit = { + val lit = GpuOverrides.extractLit(expr.right) + lit.map { l => + val instructions = JsonPathParser.parse(l.value.asInstanceOf[UTF8String].toString) + if (instructions.exists(JsonPathParser.fallbackCheck(_))) { + willNotWorkOnGpu("get_json_object on GPU does not support wildcard [*] in path") + } + } + } + + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = + GpuGetJsonObject(lhs, rhs) +} + case class GpuGetJsonObject(json: Expression, path: Expression) extends GpuBinaryExpressionArgsAnyScalar with ExpectsInputTypes { 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 4264233dea4..5cbad170281 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 @@ -3651,10 +3651,7 @@ object GpuOverrides extends Logging { ExprChecks.projectOnly( TypeSig.STRING, TypeSig.STRING, Seq(ParamCheck("json", TypeSig.STRING, TypeSig.STRING), ParamCheck("path", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING))), - (a, conf, p, r) => new BinaryExprMeta[GetJsonObject](a, conf, p, r) { - override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = - GpuGetJsonObject(lhs, rhs) - }), + (a, conf, p, r) => GpuGetJsonObjectMeta(a, conf, p, r)), expr[JsonToStructs]( "Returns a struct value with the given `jsonStr` and `schema`", ExprChecks.projectOnly( From f74ac0f853d47ad4dc38d37901b26ae5f2094dc6 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Mon, 25 Mar 2024 10:15:53 +0800 Subject: [PATCH 05/11] cancel xfail cases Signed-off-by: Haoyang Li --- integration_tests/src/main/python/json_matrix_test.py | 9 ++------- .../scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala | 5 +++-- .../scala/com/nvidia/spark/rapids/GpuOverrides.scala | 2 +- 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/integration_tests/src/main/python/json_matrix_test.py b/integration_tests/src/main/python/json_matrix_test.py index f36e5c7477d..f68a178036d 100644 --- a/integration_tests/src/main/python/json_matrix_test.py +++ b/integration_tests/src/main/python/json_matrix_test.py @@ -108,7 +108,6 @@ def test_from_json_allow_comments_off(std_input_path): # Off is the default so it really needs to work @allow_non_gpu(TEXT_INPUT_EXEC) -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10194') def test_get_json_object_allow_comments_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_COMMENTS_FILE, "json").selectExpr('''get_json_object(json, "$.str")''')) @@ -216,7 +215,6 @@ def test_from_json_allow_unquoted_field_names_on(std_input_path): # Off is the default so it really needs to work @allow_non_gpu(TEXT_INPUT_EXEC) -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10454') def test_get_json_object_allow_unquoted_field_names_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTE_FIELD_NAMES_FILE, "json").selectExpr('''get_json_object(json, "$.str")''')) @@ -276,7 +274,6 @@ def test_from_json_allow_numeric_leading_zeros_off(std_input_path): # Off is the default so it really needs to work @allow_non_gpu(TEXT_INPUT_EXEC) -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10454') def test_get_json_object_allow_numeric_leading_zeros_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NUMERIC_LEAD_ZEROS_FILE, "json").selectExpr('''get_json_object(json, "$.byte")''', @@ -339,7 +336,6 @@ def test_from_json_allow_nonnumeric_numbers_on(std_input_path): # Off is the default for get_json_object so we want this to work @allow_non_gpu(TEXT_INPUT_EXEC) -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10454') def test_get_json_object_allow_nonnumeric_numbers_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NONNUMERIC_NUMBERS_FILE, "json").selectExpr('''get_json_object(json, "$.float")''', @@ -396,7 +392,6 @@ def test_from_json_allow_backslash_escape_any_on(std_input_path): # Off is the default for get_json_object so we want this to work @allow_non_gpu(TEXT_INPUT_EXEC) -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10454') def test_get_json_object_allow_backslash_escape_any_off(std_input_path): assert_gpu_and_cpu_are_equal_collect( lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_BS_ESC_FILE, "json").selectExpr('''get_json_object(json, "$.str")''')) @@ -724,8 +719,8 @@ def test_from_json_strings(std_input_path, input_file): "sci_formatted_strings.json", "decimal_locale_formatted_strings.json", "single_quoted_strings.json", - pytest.param("boolean_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')), - pytest.param("invalid_ridealong_columns.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10534'))]) + "boolean_formatted.json", + "invalid_ridealong_columns.json"]) @allow_non_gpu(TEXT_INPUT_EXEC) def test_get_json_object_formats(std_input_path, input_file): assert_gpu_and_cpu_are_equal_collect( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala index 86e3a9b7d85..3bf6a54124d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala @@ -22,8 +22,9 @@ import ai.rapids.cudf.ColumnVector import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.jni.JSONUtils -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, GetJsonObject} import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.unsafe.types.UTF8String // Copied from Apache Spark org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala sealed trait PathInstruction @@ -85,7 +86,7 @@ object JsonPathParser extends RegexParsers { def fallbackCheck(instructions: List[PathInstruction]): Boolean = { // JNI kernel has a limit of 16 nested nodes, fallback to CPU if we exceed that - instructions.filterNot(_ == Subscript | Key).length > 16 + instructions.length > 16 } def unzipInstruction(instruction: PathInstruction): (String, String, Long) = { 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 5cbad170281..eaf4163a1c7 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 @@ -3651,7 +3651,7 @@ object GpuOverrides extends Logging { ExprChecks.projectOnly( TypeSig.STRING, TypeSig.STRING, Seq(ParamCheck("json", TypeSig.STRING, TypeSig.STRING), ParamCheck("path", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING))), - (a, conf, p, r) => GpuGetJsonObjectMeta(a, conf, p, r)), + (a, conf, p, r) => new GpuGetJsonObjectMeta(a, conf, p, r)), expr[JsonToStructs]( "Returns a struct value with the given `jsonStr` and `schema`", ExprChecks.projectOnly( From 02ff03a69aaf81f7431a888135515348354a356c Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Mon, 25 Mar 2024 10:35:04 +0800 Subject: [PATCH 06/11] cancel xfail cases Signed-off-by: Haoyang Li --- integration_tests/src/main/python/json_matrix_test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/json_matrix_test.py b/integration_tests/src/main/python/json_matrix_test.py index 284eae58fbf..cd98385d805 100644 --- a/integration_tests/src/main/python/json_matrix_test.py +++ b/integration_tests/src/main/python/json_matrix_test.py @@ -755,11 +755,11 @@ def test_from_json_strings(std_input_path, input_file): "sci_formatted_strings.json", "decimal_locale_formatted_strings.json", "single_quoted_strings.json", - pytest.param("boolean_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')), - pytest.param("invalid_ridealong_columns.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10534')), + "boolean_formatted.json", + "invalid_ridealong_columns.json", pytest.param("int_array_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')), - pytest.param("int_struct_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')), - pytest.param("int_mixed_array_struct_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218'))]) + "int_struct_formatted.json", + "int_mixed_array_struct_formatted.json"]) @allow_non_gpu(TEXT_INPUT_EXEC) def test_get_json_object_formats(std_input_path, input_file): assert_gpu_and_cpu_are_equal_collect( From bea3b4559a7feb6af1c0e3a1eda7e13efffc5e5e Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Mon, 25 Mar 2024 13:58:51 +0800 Subject: [PATCH 07/11] generated and modified docs Signed-off-by: Haoyang Li --- docs/compatibility.md | 31 +- .../src/main/python/json_matrix_test.py | 4 +- tools/generated_files/312/supportedExprs.csv | 6 +- tools/generated_files/313/supportedExprs.csv | 6 +- tools/generated_files/320/supportedExprs.csv | 6 +- tools/generated_files/321/supportedExprs.csv | 6 +- .../generated_files/321cdh/supportedExprs.csv | 6 +- tools/generated_files/322/supportedExprs.csv | 6 +- tools/generated_files/323/supportedExprs.csv | 6 +- tools/generated_files/324/supportedExprs.csv | 6 +- tools/generated_files/330/supportedExprs.csv | 6 +- .../generated_files/330cdh/supportedExprs.csv | 6 +- tools/generated_files/331/supportedExprs.csv | 6 +- tools/generated_files/332/supportedExprs.csv | 6 +- .../generated_files/332cdh/supportedExprs.csv | 6 +- tools/generated_files/333/supportedExprs.csv | 6 +- tools/generated_files/334/supportedExprs.csv | 6 +- tools/generated_files/340/supportedExprs.csv | 6 +- tools/generated_files/342/supportedExprs.csv | 6 +- tools/generated_files/350/supportedExecs.csv | 2 +- tools/generated_files/350/supportedExprs.csv | 6 +- tools/generated_files/351/operatorsScore.csv | 287 +++++++ .../351/supportedDataSource.csv | 13 + tools/generated_files/351/supportedExecs.csv | 57 ++ tools/generated_files/351/supportedExprs.csv | 772 ++++++++++++++++++ 25 files changed, 1193 insertions(+), 81 deletions(-) create mode 100644 tools/generated_files/351/operatorsScore.csv create mode 100644 tools/generated_files/351/supportedDataSource.csv create mode 100644 tools/generated_files/351/supportedExecs.csv create mode 100644 tools/generated_files/351/supportedExprs.csv diff --git a/docs/compatibility.md b/docs/compatibility.md index b482ac70ffb..3995aa9de1b 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -439,31 +439,12 @@ Known issues are: ### get_json_object -The `GetJsonObject` operator takes a JSON formatted string and a JSON path string as input. The -code base for this is currently separate from GPU parsing of JSON for files and `FromJsonObject`. -Because of this the results can be different from each other. Because of several incompatibilities -and bugs in the GPU version of `GetJsonObject` it will be on the CPU by default. If you are -aware of the current limitations with the GPU version, you might see a significant performance -speedup if you enable it by setting `spark.rapids.sql.expression.GetJsonObject` to `true`. - -The following is a list of known differences. - * [No input validation](https://github.com/NVIDIA/spark-rapids/issues/10218). If the input string - is not valid JSON Apache Spark returns a null result, but ours will still try to find a match. - * [Escapes are not properly processed for Strings](https://github.com/NVIDIA/spark-rapids/issues/10196). - When returning a result for a quoted string Apache Spark will remove the quotes and replace - any escape sequences with the proper characters. The escape sequence processing does not happen - on the GPU. - * [Invalid JSON paths could throw exceptions](https://github.com/NVIDIA/spark-rapids/issues/10212) - If a JSON path is not valid Apache Spark returns a null result, but ours may throw an exception - and fail the query. - * [Non-string output is not normalized](https://github.com/NVIDIA/spark-rapids/issues/10218) - When returning a result for things other than strings, a number of things are normalized by - Apache Spark, but are not normalized by the GPU, like removing unnecessary white space, - parsing and then serializing floating point numbers, turning single quotes to double quotes, - and removing unneeded escapes for single quotes. - -The following is a list of bugs in either the GPU version or arguably in Apache Spark itself. - * https://github.com/NVIDIA/spark-rapids/issues/10219 non-matching quotes in quoted strings +Known issue: +- [Non-string output is not normalized](https://github.com/NVIDIA/spark-rapids/issues/10218) + When returning a result for things other than strings, a number of things are normalized by + Apache Spark, but are not normalized by the GPU, like removing unnecessary white space, + parsing and then serializing floating point numbers, turning single quotes to double quotes, + and removing unneeded escapes for single quotes. ## Avro diff --git a/integration_tests/src/main/python/json_matrix_test.py b/integration_tests/src/main/python/json_matrix_test.py index cd98385d805..a36af746a41 100644 --- a/integration_tests/src/main/python/json_matrix_test.py +++ b/integration_tests/src/main/python/json_matrix_test.py @@ -60,7 +60,9 @@ def read_json_as_text(spark, data_path, column_name): 'spark.rapids.sql.expression.JsonToStructs': 'true', 'spark.rapids.sql.json.read.float.enabled': 'true', 'spark.rapids.sql.json.read.double.enabled': 'true', - 'spark.rapids.sql.json.read.decimal.enabled': 'true' + 'spark.rapids.sql.json.read.decimal.enabled': 'true', + 'spark.rapids.sql.json.read.decimal.enabled': 'true', + 'spark.rapids.sql.json.read.mixedTypesAsString.enabled': 'true' } _enable_json_tuple_conf = { diff --git a/tools/generated_files/312/supportedExprs.csv b/tools/generated_files/312/supportedExprs.csv index 52cd9957729..b3b74b9fd5e 100644 --- a/tools/generated_files/312/supportedExprs.csv +++ b/tools/generated_files/312/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS diff --git a/tools/generated_files/313/supportedExprs.csv b/tools/generated_files/313/supportedExprs.csv index 52cd9957729..b3b74b9fd5e 100644 --- a/tools/generated_files/313/supportedExprs.csv +++ b/tools/generated_files/313/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS diff --git a/tools/generated_files/320/supportedExprs.csv b/tools/generated_files/320/supportedExprs.csv index 4b1d8e23f17..6321a763243 100644 --- a/tools/generated_files/320/supportedExprs.csv +++ b/tools/generated_files/320/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/321/supportedExprs.csv b/tools/generated_files/321/supportedExprs.csv index 4b1d8e23f17..6321a763243 100644 --- a/tools/generated_files/321/supportedExprs.csv +++ b/tools/generated_files/321/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/321cdh/supportedExprs.csv b/tools/generated_files/321cdh/supportedExprs.csv index 4b1d8e23f17..6321a763243 100644 --- a/tools/generated_files/321cdh/supportedExprs.csv +++ b/tools/generated_files/321cdh/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/322/supportedExprs.csv b/tools/generated_files/322/supportedExprs.csv index 4b1d8e23f17..6321a763243 100644 --- a/tools/generated_files/322/supportedExprs.csv +++ b/tools/generated_files/322/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/323/supportedExprs.csv b/tools/generated_files/323/supportedExprs.csv index 4b1d8e23f17..6321a763243 100644 --- a/tools/generated_files/323/supportedExprs.csv +++ b/tools/generated_files/323/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/324/supportedExprs.csv b/tools/generated_files/324/supportedExprs.csv index 4b1d8e23f17..6321a763243 100644 --- a/tools/generated_files/324/supportedExprs.csv +++ b/tools/generated_files/324/supportedExprs.csv @@ -224,9 +224,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/330/supportedExprs.csv b/tools/generated_files/330/supportedExprs.csv index f86e913a309..138e290c264 100644 --- a/tools/generated_files/330/supportedExprs.csv +++ b/tools/generated_files/330/supportedExprs.csv @@ -233,9 +233,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/330cdh/supportedExprs.csv b/tools/generated_files/330cdh/supportedExprs.csv index f86e913a309..138e290c264 100644 --- a/tools/generated_files/330cdh/supportedExprs.csv +++ b/tools/generated_files/330cdh/supportedExprs.csv @@ -233,9 +233,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/331/supportedExprs.csv b/tools/generated_files/331/supportedExprs.csv index b0fa92db757..44bbe22c111 100644 --- a/tools/generated_files/331/supportedExprs.csv +++ b/tools/generated_files/331/supportedExprs.csv @@ -235,9 +235,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/332/supportedExprs.csv b/tools/generated_files/332/supportedExprs.csv index b0fa92db757..44bbe22c111 100644 --- a/tools/generated_files/332/supportedExprs.csv +++ b/tools/generated_files/332/supportedExprs.csv @@ -235,9 +235,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/332cdh/supportedExprs.csv b/tools/generated_files/332cdh/supportedExprs.csv index b0fa92db757..44bbe22c111 100644 --- a/tools/generated_files/332cdh/supportedExprs.csv +++ b/tools/generated_files/332cdh/supportedExprs.csv @@ -235,9 +235,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/333/supportedExprs.csv b/tools/generated_files/333/supportedExprs.csv index b0fa92db757..44bbe22c111 100644 --- a/tools/generated_files/333/supportedExprs.csv +++ b/tools/generated_files/333/supportedExprs.csv @@ -235,9 +235,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/334/supportedExprs.csv b/tools/generated_files/334/supportedExprs.csv index b0fa92db757..44bbe22c111 100644 --- a/tools/generated_files/334/supportedExprs.csv +++ b/tools/generated_files/334/supportedExprs.csv @@ -235,9 +235,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/340/supportedExprs.csv b/tools/generated_files/340/supportedExprs.csv index 519fa951e3e..a6e8c8269c5 100644 --- a/tools/generated_files/340/supportedExprs.csv +++ b/tools/generated_files/340/supportedExprs.csv @@ -235,9 +235,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/342/supportedExprs.csv b/tools/generated_files/342/supportedExprs.csv index 519fa951e3e..a6e8c8269c5 100644 --- a/tools/generated_files/342/supportedExprs.csv +++ b/tools/generated_files/342/supportedExprs.csv @@ -235,9 +235,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/350/supportedExecs.csv b/tools/generated_files/350/supportedExecs.csv index 949a482a551..409fa3e45aa 100644 --- a/tools/generated_files/350/supportedExecs.csv +++ b/tools/generated_files/350/supportedExecs.csv @@ -18,7 +18,7 @@ AQEShuffleReadExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS HashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS,NS,NS ObjectHashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS,NS,NS SortAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS,NS,NS -InMemoryTableScanExec,NS,This is disabled by default because there could be complications when using it with AQE. For more details please check https://github.com/NVIDIA/spark-rapids/issues/10603,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,PS,PS,PS,NS,S,S +InMemoryTableScanExec,NS,This is disabled by default because there could be complications when using it with AQE with Spark-3.5.0 and Spark-3.5.1. For more details please check https://github.com/NVIDIA/spark-rapids/issues/10603,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,PS,PS,PS,NS,S,S DataWritingCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,PS,NS,S,NS,PS,PS,PS,NS,S,S ExecutedCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S WriteFilesExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S diff --git a/tools/generated_files/350/supportedExprs.csv b/tools/generated_files/350/supportedExprs.csv index fb5c9e0bba7..63628aabd8c 100644 --- a/tools/generated_files/350/supportedExprs.csv +++ b/tools/generated_files/350/supportedExprs.csv @@ -235,9 +235,9 @@ GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetJsonObject,NS,`get_json_object`,This is disabled by default because escape sequences are not processed correctly; the input is not validated; and the output is not normalized the same as Spark,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS diff --git a/tools/generated_files/351/operatorsScore.csv b/tools/generated_files/351/operatorsScore.csv new file mode 100644 index 00000000000..50b7084026c --- /dev/null +++ b/tools/generated_files/351/operatorsScore.csv @@ -0,0 +1,287 @@ +CPUOperator,Score +CoalesceExec,3.0 +CollectLimitExec,3.0 +ExpandExec,3.0 +FileSourceScanExec,3.0 +FilterExec,2.8 +GenerateExec,3.0 +GlobalLimitExec,3.0 +LocalLimitExec,3.0 +ProjectExec,3.0 +RangeExec,3.0 +SampleExec,3.0 +SortExec,8.0 +SubqueryBroadcastExec,3.0 +TakeOrderedAndProjectExec,3.0 +UnionExec,3.0 +AQEShuffleReadExec,3.0 +HashAggregateExec,4.5 +ObjectHashAggregateExec,3.0 +SortAggregateExec,3.0 +InMemoryTableScanExec,3.0 +DataWritingCommandExec,3.0 +ExecutedCommandExec,3.0 +WriteFilesExec,3.0 +AppendDataExecV1,3.0 +AtomicCreateTableAsSelectExec,3.0 +AtomicReplaceTableAsSelectExec,3.0 +BatchScanExec,3.0 +OverwriteByExpressionExecV1,3.0 +BroadcastExchangeExec,3.0 +ShuffleExchangeExec,4.2 +BroadcastHashJoinExec,5.1 +BroadcastNestedLoopJoinExec,3.0 +CartesianProductExec,3.0 +ShuffledHashJoinExec,3.0 +SortMergeJoinExec,22.7 +AggregateInPandasExec,1.2 +ArrowEvalPythonExec,1.2 +FlatMapCoGroupsInPandasExec,3.0 +FlatMapGroupsInPandasExec,1.2 +MapInPandasExec,1.2 +PythonMapInArrowExec,3.0 +WindowInPandasExec,1.2 +WindowExec,3.0 +WindowGroupLimitExec,3.0 +HiveTableScanExec,3.0 +Abs,4 +Acos,4 +Acosh,4 +Add,4 +AggregateExpression,4 +Alias,4 +And,4 +ApproximatePercentile,4 +ArrayContains,4 +ArrayExcept,4 +ArrayExists,4 +ArrayIntersect,4 +ArrayMax,4 +ArrayMin,4 +ArrayRemove,4 +ArrayRepeat,4 +ArrayTransform,4 +ArrayUnion,4 +ArraysOverlap,4 +ArraysZip,4 +Ascii,4 +Asin,4 +Asinh,4 +AtLeastNNonNulls,4 +Atan,4 +Atanh,4 +AttributeReference,4 +Average,4 +BRound,4 +BitLength,4 +BitwiseAnd,4 +BitwiseNot,4 +BitwiseOr,4 +BitwiseXor,4 +BloomFilterAggregate,4 +BloomFilterMightContain,4 +CaseWhen,4 +Cbrt,4 +Ceil,4 +CheckOverflowInTableInsert,4 +Coalesce,4 +CollectList,4 +CollectSet,4 +Concat,4 +ConcatWs,4 +Contains,4 +Conv,4 +Cos,4 +Cosh,4 +Cot,4 +Count,4 +CreateArray,4 +CreateMap,4 +CreateNamedStruct,4 +CurrentRow$,4 +DateAdd,4 +DateAddInterval,4 +DateDiff,4 +DateFormatClass,4 +DateSub,4 +DayOfMonth,4 +DayOfWeek,4 +DayOfYear,4 +DenseRank,4 +Divide,4 +DivideDTInterval,4 +DivideYMInterval,4 +DynamicPruningExpression,4 +ElementAt,4 +Empty2Null,4 +EndsWith,4 +EqualNullSafe,4 +EqualTo,4 +Exp,4 +Explode,4 +Expm1,4 +First,4 +Flatten,4 +Floor,4 +FormatNumber,4 +FromUTCTimestamp,4 +FromUnixTime,4 +GetArrayItem,4 +GetArrayStructFields,4 +GetJsonObject,4 +GetMapValue,4 +GetStructField,4 +GetTimestamp,4 +GreaterThan,4 +GreaterThanOrEqual,4 +Greatest,4 +HiveGenericUDF,4 +HiveSimpleUDF,4 +Hour,4 +Hypot,4 +If,4 +In,4 +InSet,4 +InSubqueryExec,4 +InitCap,4 +InputFileBlockLength,4 +InputFileBlockStart,4 +InputFileName,4 +IntegralDivide,4 +IsNaN,4 +IsNotNull,4 +IsNull,4 +JsonToStructs,4 +JsonTuple,4 +KnownFloatingPointNormalized,4 +KnownNotNull,4 +KnownNullable,4 +Lag,4 +LambdaFunction,4 +Last,4 +LastDay,4 +Lead,4 +Least,4 +Length,4 +LessThan,4 +LessThanOrEqual,4 +Like,4 +Literal,4 +Log,4 +Log10,4 +Log1p,4 +Log2,4 +Logarithm,4 +Lower,4 +MakeDecimal,4 +MapConcat,4 +MapEntries,4 +MapFilter,4 +MapKeys,4 +MapValues,4 +Max,4 +Md5,4 +MicrosToTimestamp,4 +MillisToTimestamp,4 +Min,4 +Minute,4 +MonotonicallyIncreasingID,4 +Month,4 +Multiply,4 +MultiplyDTInterval,4 +MultiplyYMInterval,4 +Murmur3Hash,4 +NaNvl,4 +NamedLambdaVariable,4 +NormalizeNaNAndZero,4 +Not,4 +NthValue,4 +OctetLength,4 +Or,4 +ParseUrl,4 +PercentRank,4 +Percentile,4 +PivotFirst,4 +Pmod,4 +PosExplode,4 +Pow,4 +PreciseTimestampConversion,4 +PythonUDAF,4 +PythonUDF,4 +Quarter,4 +RLike,4 +RaiseError,4 +Rand,4 +Rank,4 +RegExpExtract,4 +RegExpExtractAll,4 +RegExpReplace,4 +Remainder,4 +ReplicateRows,4 +Reverse,4 +Rint,4 +Round,4 +RoundCeil,4 +RoundFloor,4 +RowNumber,4 +ScalaUDF,4 +ScalarSubquery,4 +Second,4 +SecondsToTimestamp,4 +Sequence,4 +ShiftLeft,4 +ShiftRight,4 +ShiftRightUnsigned,4 +Signum,4 +Sin,4 +Sinh,4 +Size,4 +SortArray,4 +SortOrder,4 +SparkPartitionID,4 +SpecifiedWindowFrame,4 +Sqrt,4 +Stack,4 +StartsWith,4 +StddevPop,4 +StddevSamp,4 +StringInstr,4 +StringLPad,4 +StringLocate,4 +StringRPad,4 +StringRepeat,4 +StringReplace,4 +StringSplit,4 +StringToMap,4 +StringTranslate,4 +StringTrim,4 +StringTrimLeft,4 +StringTrimRight,4 +StructsToJson,4 +Substring,4 +SubstringIndex,4 +Subtract,4 +Sum,4 +Tan,4 +Tanh,4 +TimeAdd,4 +ToDegrees,4 +ToRadians,4 +ToUTCTimestamp,4 +ToUnixTimestamp,4 +TransformKeys,4 +TransformValues,4 +UnaryMinus,4 +UnaryPositive,4 +UnboundedFollowing$,4 +UnboundedPreceding$,4 +UnixTimestamp,4 +UnscaledValue,4 +Upper,4 +VariancePop,4 +VarianceSamp,4 +WeekDay,4 +WindowExpression,4 +WindowSpecDefinition,4 +XxHash64,4 +Year,4 diff --git a/tools/generated_files/351/supportedDataSource.csv b/tools/generated_files/351/supportedDataSource.csv new file mode 100644 index 00000000000..77f30cbe1de --- /dev/null +++ b/tools/generated_files/351/supportedDataSource.csv @@ -0,0 +1,13 @@ +Format,Direction,BOOLEAN,BYTE,SHORT,INT,LONG,FLOAT,DOUBLE,DATE,TIMESTAMP,STRING,DECIMAL,NULL,BINARY,CALENDAR,ARRAY,MAP,STRUCT,UDT,DAYTIME,YEARMONTH +Avro,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO +CSV,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,NA,NA,NA,NA,NA,NA +Delta,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S +Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS +HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S +JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO +ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA +ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S +Parquet,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/351/supportedExecs.csv b/tools/generated_files/351/supportedExecs.csv new file mode 100644 index 00000000000..409fa3e45aa --- /dev/null +++ b/tools/generated_files/351/supportedExecs.csv @@ -0,0 +1,57 @@ +Exec,Supported,Notes,Params,BOOLEAN,BYTE,SHORT,INT,LONG,FLOAT,DOUBLE,DATE,TIMESTAMP,STRING,DECIMAL,NULL,BINARY,CALENDAR,ARRAY,MAP,STRUCT,UDT,DAYTIME,YEARMONTH +CoalesceExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +CollectLimitExec,NS,This is disabled by default because Collect Limit replacement can be slower on the GPU; if huge number of rows in a batch it could help by limiting the number of rows transferred from GPU to CPU,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ExpandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +FileSourceScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +FilterExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +GenerateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +GlobalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S +SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +UnionExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +AQEShuffleReadExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +HashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS,NS,NS +ObjectHashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS,NS,NS +SortAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS,NS,NS +InMemoryTableScanExec,NS,This is disabled by default because there could be complications when using it with AQE with Spark-3.5.0 and Spark-3.5.1. For more details please check https://github.com/NVIDIA/spark-rapids/issues/10603,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,PS,PS,PS,NS,S,S +DataWritingCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,PS,NS,S,NS,PS,PS,PS,NS,S,S +ExecutedCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +WriteFilesExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AppendDataExecV1,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +AtomicCreateTableAsSelectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +AtomicReplaceTableAsSelectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +BatchScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +OverwriteByExpressionExecV1,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +BroadcastExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +ShuffleExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +BroadcastHashJoinExec,S,None,leftKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +BroadcastHashJoinExec,S,None,rightKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +BroadcastHashJoinExec,S,None,condition,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BroadcastHashJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +BroadcastNestedLoopJoinExec,S,None,condition(A non-inner join only is supported if the condition expression can be converted to a GPU AST expression),S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BroadcastNestedLoopJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CartesianProductExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +ShuffledHashJoinExec,S,None,leftKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +ShuffledHashJoinExec,S,None,rightKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +ShuffledHashJoinExec,S,None,condition,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShuffledHashJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +SortMergeJoinExec,S,None,leftKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +SortMergeJoinExec,S,None,rightKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +SortMergeJoinExec,S,None,condition,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SortMergeJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +AggregateInPandasExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +ArrowEvalPythonExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +FlatMapCoGroupsInPandasExec,NS,This is disabled by default because Performance is not ideal with many small groups,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +FlatMapGroupsInPandasExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +MapInPandasExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonMapInArrowExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +WindowInPandasExec,NS,This is disabled by default because it only supports row based frame for now,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,NS,NS,NS,NS +WindowExec,S,None,partitionSpec,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS +WindowExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +WindowGroupLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +HiveTableScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS diff --git a/tools/generated_files/351/supportedExprs.csv b/tools/generated_files/351/supportedExprs.csv new file mode 100644 index 00000000000..63628aabd8c --- /dev/null +++ b/tools/generated_files/351/supportedExprs.csv @@ -0,0 +1,772 @@ +Expression,Supported,SQL Func,Notes,Context,Params,BOOLEAN,BYTE,SHORT,INT,LONG,FLOAT,DOUBLE,DATE,TIMESTAMP,STRING,DECIMAL,NULL,BINARY,CALENDAR,ARRAY,MAP,STRUCT,UDT,DAYTIME,YEARMONTH +Abs,S,`abs`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,S +Abs,S,`abs`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,S +Abs,S,`abs`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NS,NS +Abs,S,`abs`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NS,NS +Acos,S,`acos`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acos,S,`acos`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acos,S,`acos`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acos,S,`acos`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acosh,S,`acosh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acosh,S,`acosh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acosh,S,`acosh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acosh,S,`acosh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Add,S,`+`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Add,S,`+`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Add,S,`+`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Add,S,`+`,None,AST,lhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Add,S,`+`,None,AST,rhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Add,S,`+`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Alias,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +Alias,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +Alias,S, ,None,AST,input,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,S,S +Alias,S, ,None,AST,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,S,S +And,S,`and`,None,project,lhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,project,rhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,AST,lhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,AST,rhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayContains,S,`array_contains`,None,project,array,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayContains,S,`array_contains`,None,project,key,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +ArrayContains,S,`array_contains`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayExcept,S,`array_except`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array1,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayExcept,S,`array_except`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array2,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayExcept,S,`array_except`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayExists,S,`exists`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayExists,S,`exists`,None,project,function,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayExists,S,`exists`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayIntersect,S,`array_intersect`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array1,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayIntersect,S,`array_intersect`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array2,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayIntersect,S,`array_intersect`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayMax,S,`array_max`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayMax,S,`array_max`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +ArrayMin,S,`array_min`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayMin,S,`array_min`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +ArrayRemove,S,`array_remove`,None,project,array,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,PS,NS,NS,NS,NS,NS +ArrayRemove,S,`array_remove`,None,project,element,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ArrayRemove,S,`array_remove`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayRepeat,S,`array_repeat`,None,project,left,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ArrayRepeat,S,`array_repeat`,None,project,right,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayRepeat,S,`array_repeat`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayTransform,S,`transform`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayTransform,S,`transform`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ArrayTransform,S,`transform`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayUnion,S,`array_union`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array1,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayUnion,S,`array_union`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array2,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayUnion,S,`array_union`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArraysOverlap,S,`arrays_overlap`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array1,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArraysOverlap,S,`arrays_overlap`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array2,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArraysOverlap,S,`arrays_overlap`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArraysZip,S,`arrays_zip`,None,project,children,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArraysZip,S,`arrays_zip`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Ascii,NS,`ascii`,This is disabled by default because it only supports strings starting with ASCII or Latin-1 characters after Spark 3.2.3; 3.3.1 and 3.4.0. Otherwise the results will not match the CPU.,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Ascii,NS,`ascii`,This is disabled by default because it only supports strings starting with ASCII or Latin-1 characters after Spark 3.2.3; 3.3.1 and 3.4.0. Otherwise the results will not match the CPU.,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asin,S,`asin`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asin,S,`asin`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asin,S,`asin`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asin,S,`asin`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asinh,S,`asinh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asinh,S,`asinh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asinh,S,`asinh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asinh,S,`asinh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AtLeastNNonNulls,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +AtLeastNNonNulls,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atan,S,`atan`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atan,S,`atan`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atan,S,`atan`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atan,S,`atan`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atanh,S,`atanh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atanh,S,`atanh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atanh,S,`atanh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atanh,S,`atanh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AttributeReference,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +AttributeReference,S, ,None,AST,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,S,S +BRound,S,`bround`,None,project,value,NA,S,S,S,S,PS,PS,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +BRound,S,`bround`,None,project,scale,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BRound,S,`bround`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitLength,S,`bit_length`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +BitLength,S,`bit_length`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,project,lhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,project,rhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,project,result,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,AST,lhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,AST,rhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,AST,result,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseNot,S,`~`,None,project,input,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseNot,S,`~`,None,project,result,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseNot,S,`~`,None,AST,input,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseNot,S,`~`,None,AST,result,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,project,lhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,project,rhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,project,result,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,AST,lhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,AST,rhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,AST,result,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,project,lhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,project,rhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,project,result,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,AST,lhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,AST,rhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,AST,result,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterMightContain,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA +BloomFilterMightContain,S, ,None,project,rhs,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterMightContain,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +CaseWhen,S,`when`,None,project,predicate,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +CaseWhen,S,`when`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CaseWhen,S,`when`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Cbrt,S,`cbrt`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cbrt,S,`cbrt`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cbrt,S,`cbrt`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cbrt,S,`cbrt`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Ceil,S, ,None,project,input,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Ceil,S, ,None,project,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +CheckOverflowInTableInsert,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +CheckOverflowInTableInsert,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Coalesce,S,`coalesce`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +Coalesce,S,`coalesce`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +Concat,S,`concat`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,PS,NA,NA,NA,NA,NA +Concat,S,`concat`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,PS,NA,NA,NA,NA,NA +ConcatWs,S,`concat_ws`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +ConcatWs,S,`concat_ws`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Contains,S, ,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Contains,S, ,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Contains,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Conv,NS,`conv`,This is disabled by default because GPU implementation is incomplete. We currently only support from/to_base values of 10 and 16. We fall back on CPU if the signed conversion is signalled via a negative to_base. GPU implementation does not check for an 64-bit signed/unsigned int overflow when performing the conversion to return `FFFFFFFFFFFFFFFF` or `18446744073709551615` or to throw an error in the ANSI mode. It is safe to enable if the overflow is not possible or detected externally. For instance decimal strings not longer than 18 characters / hexadecimal strings not longer than 15 characters disregarding the sign cannot cause an overflow. ,project,num,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Conv,NS,`conv`,This is disabled by default because GPU implementation is incomplete. We currently only support from/to_base values of 10 and 16. We fall back on CPU if the signed conversion is signalled via a negative to_base. GPU implementation does not check for an 64-bit signed/unsigned int overflow when performing the conversion to return `FFFFFFFFFFFFFFFF` or `18446744073709551615` or to throw an error in the ANSI mode. It is safe to enable if the overflow is not possible or detected externally. For instance decimal strings not longer than 18 characters / hexadecimal strings not longer than 15 characters disregarding the sign cannot cause an overflow. ,project,from_base,NA,PS,PS,PS,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Conv,NS,`conv`,This is disabled by default because GPU implementation is incomplete. We currently only support from/to_base values of 10 and 16. We fall back on CPU if the signed conversion is signalled via a negative to_base. GPU implementation does not check for an 64-bit signed/unsigned int overflow when performing the conversion to return `FFFFFFFFFFFFFFFF` or `18446744073709551615` or to throw an error in the ANSI mode. It is safe to enable if the overflow is not possible or detected externally. For instance decimal strings not longer than 18 characters / hexadecimal strings not longer than 15 characters disregarding the sign cannot cause an overflow. ,project,to_base,NA,PS,PS,PS,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Conv,NS,`conv`,This is disabled by default because GPU implementation is incomplete. We currently only support from/to_base values of 10 and 16. We fall back on CPU if the signed conversion is signalled via a negative to_base. GPU implementation does not check for an 64-bit signed/unsigned int overflow when performing the conversion to return `FFFFFFFFFFFFFFFF` or `18446744073709551615` or to throw an error in the ANSI mode. It is safe to enable if the overflow is not possible or detected externally. For instance decimal strings not longer than 18 characters / hexadecimal strings not longer than 15 characters disregarding the sign cannot cause an overflow. ,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cos,S,`cos`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cos,S,`cos`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cos,S,`cos`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cos,S,`cos`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cosh,S,`cosh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cosh,S,`cosh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cosh,S,`cosh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cosh,S,`cosh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cot,S,`cot`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cot,S,`cot`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cot,S,`cot`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cot,S,`cot`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +CreateArray,S,`array`,None,project,arg,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,NS,PS,NS,NS,NS +CreateArray,S,`array`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CreateMap,S,`map`,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,S,NA,NA,PS,NA,PS,NA,NA,NA +CreateMap,S,`map`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NA,NA,PS,PS,PS,NA,NA,NA +CreateNamedStruct,S,`named_struct`; `struct`,None,project,name,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +CreateNamedStruct,S,`named_struct`; `struct`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CreateNamedStruct,S,`named_struct`; `struct`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA +CurrentRow$,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +DateAdd,S,`date_add`; `dateadd`,None,project,startDate,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateAdd,S,`date_add`; `dateadd`,None,project,days,NA,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateAdd,S,`date_add`; `dateadd`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateAddInterval,S, ,None,project,start,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateAddInterval,S, ,None,project,interval,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA +DateAddInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateDiff,S,`date_diff`; `datediff`,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateDiff,S,`date_diff`; `datediff`,None,project,rhs,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateDiff,S,`date_diff`; `datediff`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateFormatClass,S,`date_format`,None,project,timestamp,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateFormatClass,S,`date_format`,None,project,strfmt,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateFormatClass,S,`date_format`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateSub,S,`date_sub`,None,project,startDate,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateSub,S,`date_sub`,None,project,days,NA,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateSub,S,`date_sub`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfMonth,S,`day`; `dayofmonth`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfMonth,S,`day`; `dayofmonth`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfWeek,S,`dayofweek`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfWeek,S,`dayofweek`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfYear,S,`dayofyear`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfYear,S,`dayofyear`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DenseRank,S,`dense_rank`,None,window,ordering,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +DenseRank,S,`dense_rank`,None,window,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Divide,S,`/`,None,project,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Divide,S,`/`,None,project,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Divide,S,`/`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +DivideDTInterval,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA +DivideDTInterval,S, ,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +DivideDTInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA +DivideYMInterval,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S +DivideYMInterval,S, ,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +DivideYMInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S +DynamicPruningExpression,S, ,None,project,input,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DynamicPruningExpression,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +ElementAt,S,`element_at`,None,project,array/map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA,NA,NA +ElementAt,S,`element_at`,None,project,index/key,PS,PS,PS,S,PS,PS,PS,PS,PS,PS,PS,NS,NS,NS,NS,NS,NS,NS,NS,NS +ElementAt,S,`element_at`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Empty2Null,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Empty2Null,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EndsWith,S, ,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EndsWith,S, ,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EndsWith,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EqualNullSafe,S,`<=>`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +EqualNullSafe,S,`<=>`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +EqualNullSafe,S,`<=>`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EqualTo,S,`==`; `=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +EqualTo,S,`==`; `=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +EqualTo,S,`==`; `=`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EqualTo,S,`==`; `=`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +EqualTo,S,`==`; `=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +EqualTo,S,`==`; `=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Exp,S,`exp`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Exp,S,`exp`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Exp,S,`exp`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Exp,S,`exp`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Explode,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA,NA,NA +Explode,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Expm1,S,`expm1`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Expm1,S,`expm1`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Expm1,S,`expm1`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Expm1,S,`expm1`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Flatten,S,`flatten`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Flatten,S,`flatten`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Floor,S, ,None,project,input,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Floor,S, ,None,project,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,x,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,d,NA,NA,NA,PS,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timestamp,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timezone,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUTCTimestamp,S,`from_utc_timestamp`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUnixTime,S,`from_unixtime`,None,project,sec,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUnixTime,S,`from_unixtime`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUnixTime,S,`from_unixtime`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetArrayItem,S, ,None,project,array,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS +GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +GetStructField,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA +GetStructField,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +GetTimestamp,S, ,None,project,timeExp,NA,NA,NA,NA,NA,NA,NA,S,PS,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetTimestamp,S, ,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetTimestamp,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GreaterThan,S,`>`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +GreaterThan,S,`>`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +GreaterThan,S,`>`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GreaterThan,S,`>`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +GreaterThan,S,`>`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +GreaterThan,S,`>`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GreaterThanOrEqual,S,`>=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +GreaterThanOrEqual,S,`>=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +GreaterThanOrEqual,S,`>=`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GreaterThanOrEqual,S,`>=`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Hypot,S,`hypot`,None,project,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Hypot,S,`hypot`,None,project,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Hypot,S,`hypot`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +If,S,`if`,None,project,predicate,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +If,S,`if`,None,project,trueValue,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +If,S,`if`,None,project,falseValue,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +If,S,`if`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +In,S,`in`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +In,S,`in`,None,project,list,PS,PS,PS,PS,PS,PS,PS,PS,PS,PS,PS,NS,NS,NS,NS,NA,NS,NS,NA,NA +In,S,`in`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InSet,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +InSet,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InitCap,S,`initcap`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InitCap,S,`initcap`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InputFileBlockLength,S,`input_file_block_length`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InputFileBlockStart,S,`input_file_block_start`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InputFileName,S,`input_file_name`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IntegralDivide,S,`div`,None,project,lhs,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +IntegralDivide,S,`div`,None,project,rhs,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +IntegralDivide,S,`div`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IsNaN,S,`isnan`,None,project,input,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IsNaN,S,`isnan`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS +IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS +IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA +JsonTuple,NS,`json_tuple`,This is disabled by default because JsonTuple on the GPU does not support all of the normalization that the CPU supports.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonTuple,NS,`json_tuple`,This is disabled by default because JsonTuple on the GPU does not support all of the normalization that the CPU supports.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonTuple,NS,`json_tuple`,This is disabled by default because JsonTuple on the GPU does not support all of the normalization that the CPU supports.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +KnownFloatingPointNormalized,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +KnownFloatingPointNormalized,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +KnownNotNull,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS,NS,NS +KnownNotNull,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS,NS,NS +KnownNullable,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +KnownNullable,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Lag,S,`lag`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Lag,S,`lag`,None,window,offset,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lag,S,`lag`,None,window,default,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Lag,S,`lag`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +LambdaFunction,S, ,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +LambdaFunction,S, ,None,project,arguments,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +LambdaFunction,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +LastDay,S,`last_day`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LastDay,S,`last_day`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lead,S,`lead`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Lead,S,`lead`,None,window,offset,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lead,S,`lead`,None,window,default,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Lead,S,`lead`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Least,S,`least`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Least,S,`least`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Length,S,`char_length`; `character_length`; `len`; `length`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +Length,S,`char_length`; `character_length`; `len`; `length`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LessThan,S,`<`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +LessThan,S,`<`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +LessThan,S,`<`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LessThan,S,`<`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +LessThan,S,`<`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +LessThan,S,`<`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LessThanOrEqual,S,`<=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +LessThanOrEqual,S,`<=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +LessThanOrEqual,S,`<=`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LessThanOrEqual,S,`<=`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +LessThanOrEqual,S,`<=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +LessThanOrEqual,S,`<=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Like,S,`like`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Like,S,`like`,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Like,S,`like`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Literal,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,S,S +Literal,S, ,None,AST,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +Log,S,`ln`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log,S,`ln`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log10,S,`log10`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log10,S,`log10`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log1p,S,`log1p`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log1p,S,`log1p`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log2,S,`log2`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log2,S,`log2`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Logarithm,S,`log`,None,project,value,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Logarithm,S,`log`,None,project,base,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Logarithm,S,`log`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lower,S,`lcase`; `lower`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lower,S,`lcase`; `lower`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MakeDecimal,S, ,None,project,input,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MakeDecimal,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA +MapConcat,S,`map_concat`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapConcat,S,`map_concat`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapEntries,S,`map_entries`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapEntries,S,`map_entries`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +MapFilter,S,`map_filter`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapFilter,S,`map_filter`,None,project,function,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MapFilter,S,`map_filter`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapKeys,S,`map_keys`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapKeys,S,`map_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +MapValues,S,`map_values`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapValues,S,`map_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Md5,S,`md5`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +Md5,S,`md5`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MicrosToTimestamp,S,`timestamp_micros`,None,project,input,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MicrosToTimestamp,S,`timestamp_micros`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MillisToTimestamp,S,`timestamp_millis`,None,project,input,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MillisToTimestamp,S,`timestamp_millis`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Minute,S,`minute`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,AST,lhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,AST,rhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +MultiplyDTInterval,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA +MultiplyDTInterval,S, ,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +MultiplyDTInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA +MultiplyYMInterval,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S +MultiplyYMInterval,S, ,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +MultiplyYMInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S +Murmur3Hash,S,`hash`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Murmur3Hash,S,`hash`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NaNvl,S,`nanvl`,None,project,lhs,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NaNvl,S,`nanvl`,None,project,rhs,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NaNvl,S,`nanvl`,None,project,result,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NamedLambdaVariable,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +Not,S,`!`; `not`,None,project,input,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Not,S,`!`; `not`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Not,S,`!`; `not`,None,AST,input,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Not,S,`!`; `not`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NthValue,S,`nth_value`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +NthValue,S,`nth_value`,None,window,offset,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NthValue,S,`nth_value`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +OctetLength,S,`octet_length`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +OctetLength,S,`octet_length`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Or,S,`or`,None,project,lhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Or,S,`or`,None,project,rhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Or,S,`or`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Or,S,`or`,None,AST,lhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Or,S,`or`,None,AST,rhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Or,S,`or`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ParseUrl,S,`parse_url`,None,project,url,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ParseUrl,S,`parse_url`,None,project,partToExtract,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ParseUrl,S,`parse_url`,None,project,key,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ParseUrl,S,`parse_url`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +PercentRank,S,`percent_rank`,None,window,ordering,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +PercentRank,S,`percent_rank`,None,window,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pmod,S,`pmod`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pmod,S,`pmod`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pmod,S,`pmod`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +PosExplode,S,`posexplode_outer`; `posexplode`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA,NA,NA +PosExplode,S,`posexplode_outer`; `posexplode`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,project,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,project,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,AST,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,AST,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +PreciseTimestampConversion,S, ,None,project,input,NA,NA,NA,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +PreciseTimestampConversion,S, ,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +PythonUDAF,S, ,None,aggregation,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDAF,S, ,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDAF,S, ,None,reduction,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDAF,S, ,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDAF,S, ,None,window,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDAF,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDAF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDAF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDF,S, ,None,aggregation,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDF,S, ,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDF,S, ,None,reduction,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDF,S, ,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDF,S, ,None,window,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDF,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +Quarter,S,`quarter`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Quarter,S,`quarter`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RLike,S,`regexp_like`; `regexp`; `rlike`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RLike,S,`regexp_like`; `regexp`; `rlike`,None,project,regexp,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RLike,S,`regexp_like`; `regexp`; `rlike`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RaiseError,S,`raise_error`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RaiseError,S,`raise_error`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +Rand,S,`rand`; `random`,None,project,seed,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rand,S,`rand`; `random`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rank,S,`rank`,None,window,ordering,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +Rank,S,`rank`,None,window,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtract,S,`regexp_extract`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtract,S,`regexp_extract`,None,project,regexp,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtract,S,`regexp_extract`,None,project,idx,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtract,S,`regexp_extract`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtractAll,S,`regexp_extract_all`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtractAll,S,`regexp_extract_all`,None,project,regexp,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtractAll,S,`regexp_extract_all`,None,project,idx,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtractAll,S,`regexp_extract_all`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,regex,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,pos,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,rep,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Remainder,S,`%`; `mod`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Remainder,S,`%`; `mod`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Remainder,S,`%`; `mod`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +ReplicateRows,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +ReplicateRows,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Reverse,S,`reverse`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Reverse,S,`reverse`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Rint,S,`rint`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rint,S,`rint`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rint,S,`rint`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rint,S,`rint`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Round,S,`round`,None,project,value,NA,S,S,S,S,PS,PS,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Round,S,`round`,None,project,scale,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Round,S,`round`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundCeil,S, ,None,project,value,NA,S,S,S,S,PS,PS,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundCeil,S, ,None,project,scale,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundCeil,S, ,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundFloor,S, ,None,project,value,NA,S,S,S,S,PS,PS,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundFloor,S, ,None,project,scale,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundFloor,S, ,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RowNumber,S,`row_number`,None,window,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ScalaUDF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +ScalaUDF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +Second,S,`second`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Second,S,`second`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SecondsToTimestamp,S,`timestamp_seconds`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +SecondsToTimestamp,S,`timestamp_seconds`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sequence,S,`sequence`,None,project,start,NA,S,S,S,S,NA,NA,NS,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sequence,S,`sequence`,None,project,stop,NA,S,S,S,S,NA,NA,NS,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sequence,S,`sequence`,None,project,step,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA +Sequence,S,`sequence`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ShiftLeft,S,`shiftleft`,None,project,value,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftLeft,S,`shiftleft`,None,project,amount,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftLeft,S,`shiftleft`,None,project,result,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRight,S,`shiftright`,None,project,value,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRight,S,`shiftright`,None,project,amount,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRight,S,`shiftright`,None,project,result,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRightUnsigned,S,`shiftrightunsigned`,None,project,value,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRightUnsigned,S,`shiftrightunsigned`,None,project,amount,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRightUnsigned,S,`shiftrightunsigned`,None,project,result,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Signum,S,`sign`; `signum`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Signum,S,`sign`; `signum`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sin,S,`sin`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sin,S,`sin`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sin,S,`sin`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sin,S,`sin`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sinh,S,`sinh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sinh,S,`sinh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sinh,S,`sinh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sinh,S,`sinh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Size,S,`cardinality`; `size`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA,NA,NA +Size,S,`cardinality`; `size`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SortArray,S,`sort_array`,None,project,array,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +SortArray,S,`sort_array`,None,project,ascendingOrder,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SortArray,S,`sort_array`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +SortOrder,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +SortOrder,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +SparkPartitionID,S,`spark_partition_id`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SpecifiedWindowFrame,S, ,None,project,lower,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,S,NA,NA,NA,NA,S,NS +SpecifiedWindowFrame,S, ,None,project,upper,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,S,NA,NA,NA,NA,S,NS +SpecifiedWindowFrame,S, ,None,project,result,NA,S,S,S,S,NS,NS,NA,NA,NA,NS,NA,NA,S,NA,NA,NA,NA,S,NS +Sqrt,S,`sqrt`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sqrt,S,`sqrt`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sqrt,S,`sqrt`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sqrt,S,`sqrt`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Stack,S,`stack`,None,project,n,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Stack,S,`stack`,None,project,expr,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +Stack,S,`stack`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +StartsWith,S, ,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StartsWith,S, ,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StartsWith,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringInstr,S,`instr`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringInstr,S,`instr`,None,project,substr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringInstr,S,`instr`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLPad,S, ,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLPad,S, ,None,project,len,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLPad,S, ,None,project,pad,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLPad,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLocate,S,`locate`; `position`,None,project,substr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLocate,S,`locate`; `position`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLocate,S,`locate`; `position`,None,project,start,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLocate,S,`locate`; `position`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRPad,S, ,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRPad,S, ,None,project,len,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRPad,S, ,None,project,pad,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRPad,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRepeat,S,`repeat`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRepeat,S,`repeat`,None,project,repeatTimes,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRepeat,S,`repeat`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringReplace,S,`replace`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringReplace,S,`replace`,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringReplace,S,`replace`,None,project,replace,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringReplace,S,`replace`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringSplit,S,`split`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringSplit,S,`split`,None,project,regexp,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringSplit,S,`split`,None,project,limit,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringSplit,S,`split`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +StringToMap,S,`str_to_map`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringToMap,S,`str_to_map`,None,project,pairDelim,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringToMap,S,`str_to_map`,None,project,keyValueDelim,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringToMap,S,`str_to_map`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA +StringTranslate,S,`translate`,This is not 100% compatible with the Spark version because the GPU implementation supports all unicode code points. In Spark versions < 3.2.0; translate() does not support unicode characters with code point >= U+10000 (See SPARK-34094),project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTranslate,S,`translate`,This is not 100% compatible with the Spark version because the GPU implementation supports all unicode code points. In Spark versions < 3.2.0; translate() does not support unicode characters with code point >= U+10000 (See SPARK-34094),project,from,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTranslate,S,`translate`,This is not 100% compatible with the Spark version because the GPU implementation supports all unicode code points. In Spark versions < 3.2.0; translate() does not support unicode characters with code point >= U+10000 (See SPARK-34094),project,to,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTranslate,S,`translate`,This is not 100% compatible with the Spark version because the GPU implementation supports all unicode code points. In Spark versions < 3.2.0; translate() does not support unicode characters with code point >= U+10000 (See SPARK-34094),project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrim,S,`trim`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrim,S,`trim`,None,project,trimStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrim,S,`trim`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimLeft,S,`ltrim`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimLeft,S,`ltrim`,None,project,trimStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimLeft,S,`ltrim`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimRight,S,`rtrim`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimRight,S,`rtrim`,None,project,trimStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimRight,S,`rtrim`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StructsToJson,NS,`to_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,struct,S,S,S,S,S,S,S,S,PS,S,S,NA,NA,NA,PS,PS,PS,NA,NA,NA +StructsToJson,NS,`to_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Substring,S,`substr`; `substring`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +Substring,S,`substr`; `substring`,None,project,pos,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Substring,S,`substr`; `substring`,None,project,len,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Substring,S,`substr`; `substring`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +SubstringIndex,S,`substring_index`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SubstringIndex,S,`substring_index`,None,project,delim,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SubstringIndex,S,`substring_index`,None,project,count,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SubstringIndex,S,`substring_index`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Subtract,S,`-`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Subtract,S,`-`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Subtract,S,`-`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Subtract,S,`-`,None,AST,lhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Subtract,S,`-`,None,AST,rhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Subtract,S,`-`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Tan,S,`tan`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tan,S,`tan`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tan,S,`tan`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tan,S,`tan`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tanh,S,`tanh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tanh,S,`tanh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tanh,S,`tanh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tanh,S,`tanh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TimeAdd,S, ,None,project,start,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TimeAdd,S, ,None,project,interval,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,S,NA +TimeAdd,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToDegrees,S,`degrees`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToDegrees,S,`degrees`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToRadians,S,`radians`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToRadians,S,`radians`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUTCTimestamp,S,`to_utc_timestamp`,None,project,timestamp,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUTCTimestamp,S,`to_utc_timestamp`,None,project,timezone,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUTCTimestamp,S,`to_utc_timestamp`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUnixTimestamp,S,`to_unix_timestamp`,None,project,timeExp,NA,NA,NA,NA,NA,NA,NA,S,PS,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUnixTimestamp,S,`to_unix_timestamp`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUnixTimestamp,S,`to_unix_timestamp`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TransformKeys,S,`transform_keys`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TransformKeys,S,`transform_keys`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NS,NS +TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +UnaryMinus,S,`negative`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +UnaryPositive,S,`positive`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryPositive,S,`positive`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryPositive,S,`positive`,None,AST,input,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryPositive,S,`positive`,None,AST,result,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,S,S +UnboundedFollowing$,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +UnboundedPreceding$,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +UnixTimestamp,S,`unix_timestamp`,None,project,timeExp,NA,NA,NA,NA,NA,NA,NA,S,PS,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +UnixTimestamp,S,`unix_timestamp`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +UnixTimestamp,S,`unix_timestamp`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +UnscaledValue,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA +UnscaledValue,S, ,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Upper,S,`ucase`; `upper`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Upper,S,`ucase`; `upper`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +WeekDay,S,`weekday`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +WeekDay,S,`weekday`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +WindowExpression,S, ,None,window,windowFunction,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +WindowExpression,S, ,None,window,windowSpec,NA,S,S,S,S,NS,NS,NA,NA,NA,PS,NA,NA,S,NA,NA,NA,NA,S,NS +WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS +WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS +WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AggregateExpression,S, ,None,aggregation,aggFunc,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,aggregation,filter,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AggregateExpression,S, ,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,reduction,aggFunc,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,reduction,filter,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AggregateExpression,S, ,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,window,aggFunc,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,window,filter,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AggregateExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,aggregation,input,NA,S,S,S,S,S,S,NS,NS,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,aggregation,percentage,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,aggregation,accuracy,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,aggregation,result,NA,S,S,S,S,S,S,NS,NS,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,reduction,input,NA,S,S,S,S,S,S,NS,NS,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,reduction,percentage,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,reduction,accuracy,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,reduction,result,NA,S,S,S,S,S,S,NS,NS,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA +Average,S,`avg`; `mean`,None,aggregation,input,NA,S,S,S,S,S,S,NA,NA,NA,S,S,NA,NS,NA,NA,NA,NA,NS,NS +Average,S,`avg`; `mean`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Average,S,`avg`; `mean`,None,reduction,input,NA,S,S,S,S,S,S,NA,NA,NA,S,S,NA,NS,NA,NA,NA,NA,NS,NS +Average,S,`avg`; `mean`,None,reduction,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Average,S,`avg`; `mean`,None,window,input,NA,S,S,S,S,S,S,NA,NA,NA,S,S,NA,NS,NA,NA,NA,NA,NS,NS +Average,S,`avg`; `mean`,None,window,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterAggregate,S, ,None,reduction,child,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterAggregate,S, ,None,reduction,estimatedItems,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterAggregate,S, ,None,reduction,numBits,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterAggregate,S, ,None,reduction,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +CollectList,S,`array_agg`; `collect_list`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CollectList,S,`array_agg`; `collect_list`,None,aggregation,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectList,S,`array_agg`; `collect_list`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CollectList,S,`array_agg`; `collect_list`,None,reduction,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectList,S,`array_agg`; `collect_list`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CollectList,S,`array_agg`; `collect_list`,None,window,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectSet,S,`collect_set`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +CollectSet,S,`collect_set`,None,aggregation,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectSet,S,`collect_set`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +CollectSet,S,`collect_set`,None,reduction,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectSet,S,`collect_set`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +CollectSet,S,`collect_set`,None,window,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Count,S,`count`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Count,S,`count`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Count,S,`count`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Count,S,`count`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Count,S,`count`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Count,S,`count`,None,window,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +First,S,`first_value`; `first`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Max,S,`max`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Max,S,`max`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Max,S,`max`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Max,S,`max`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Max,S,`max`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Max,S,`max`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Min,S,`min`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Min,S,`min`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Min,S,`min`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Min,S,`min`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Min,S,`min`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Min,S,`min`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Percentile,S,`percentile`,None,aggregation,input,NA,S,S,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,aggregation,percentage,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,aggregation,frequency,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,reduction,input,NA,S,S,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,reduction,percentage,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,reduction,frequency,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,reduction,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +PivotFirst,S, ,None,aggregation,pivotColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,aggregation,valueColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,reduction,pivotColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,reduction,valueColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,NS,NS,NS,NS +StddevPop,S,`stddev_pop`,None,reduction,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,reduction,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,aggregation,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,window,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,window,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,aggregation,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,reduction,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,reduction,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,window,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,window,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,aggregation,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,aggregation,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,reduction,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,reduction,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,window,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,window,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,reduction,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,reduction,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,aggregation,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,window,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,window,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,reduction,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,reduction,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,aggregation,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,window,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,window,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NormalizeNaNAndZero,S, ,None,project,input,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NormalizeNaNAndZero,S, ,None,project,result,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InSubqueryExec,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +InSubqueryExec,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ScalarSubquery,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +HiveGenericUDF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +HiveGenericUDF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +HiveSimpleUDF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +HiveSimpleUDF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS From d368eb8cc275bd60adaeef4fc547e24b2f6577ff Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Tue, 26 Mar 2024 10:49:40 +0800 Subject: [PATCH 08/11] wip Signed-off-by: Haoyang Li --- integration_tests/src/main/python/get_json_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/get_json_test.py b/integration_tests/src/main/python/get_json_test.py index ad0ab9069f9..304ba965af7 100644 --- a/integration_tests/src/main/python/get_json_test.py +++ b/integration_tests/src/main/python/get_json_test.py @@ -104,7 +104,7 @@ def test_get_json_object_spark_unit_tests(query): f.get_json_object('jsonStr', query)), conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) -@pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/10218") +# @pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/10218") def test_get_json_object_normalize_non_string_output(): schema = StructType([StructField("jsonStr", StringType())]) data = [[' { "a": "A" } '], From 310916f62663ec7b158cd7b4b933829cbfbb9a63 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Tue, 26 Mar 2024 14:13:45 +0800 Subject: [PATCH 09/11] wip Signed-off-by: Haoyang Li --- integration_tests/src/main/python/json_matrix_test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/json_matrix_test.py b/integration_tests/src/main/python/json_matrix_test.py index a36af746a41..9d5ccefc1ae 100644 --- a/integration_tests/src/main/python/json_matrix_test.py +++ b/integration_tests/src/main/python/json_matrix_test.py @@ -749,9 +749,9 @@ def test_from_json_strings(std_input_path, input_file): conf =_enable_json_to_structs_conf) @pytest.mark.parametrize('input_file', [ - pytest.param("int_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')), - pytest.param("float_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')), - pytest.param("sci_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')), + "int_formatted.json", + "float_formatted.json", + "sci_formatted.json", "int_formatted_strings.json", "float_formatted_strings.json", "sci_formatted_strings.json", @@ -759,7 +759,7 @@ def test_from_json_strings(std_input_path, input_file): "single_quoted_strings.json", "boolean_formatted.json", "invalid_ridealong_columns.json", - pytest.param("int_array_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')), + "int_array_formatted.json", "int_struct_formatted.json", "int_mixed_array_struct_formatted.json"]) @allow_non_gpu(TEXT_INPUT_EXEC) From 700cf5a3729c73adf40dcd523b1be1fdda2ed262 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Tue, 26 Mar 2024 22:44:42 +0800 Subject: [PATCH 10/11] apply jni change and remove xpass Signed-off-by: Haoyang Li --- integration_tests/src/main/python/get_json_test.py | 3 +-- .../scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala | 5 ++--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/integration_tests/src/main/python/get_json_test.py b/integration_tests/src/main/python/get_json_test.py index 304ba965af7..b0e8fcec03f 100644 --- a/integration_tests/src/main/python/get_json_test.py +++ b/integration_tests/src/main/python/get_json_test.py @@ -69,8 +69,7 @@ def test_get_json_object_single_quotes(): "$['key with spaces']", "$.store.book", "$.store.book[0]", - pytest.param("$",marks=[ - pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10218')]), + "$", "$.store.book[0].category", "$.store.basket[0][1]", "$.store.basket[0][2].b", diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala index 3bf6a54124d..3f95dec95da 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGetJsonObject.scala @@ -86,7 +86,7 @@ object JsonPathParser extends RegexParsers { def fallbackCheck(instructions: List[PathInstruction]): Boolean = { // JNI kernel has a limit of 16 nested nodes, fallback to CPU if we exceed that - instructions.length > 16 + instructions.length > 32 } def unzipInstruction(instruction: PathInstruction): (String, String, Long) = { @@ -165,8 +165,7 @@ case class GpuGetJsonObject(json: Expression, path: Expression) } match { case Some(instructions) => { val jniInstructions = JsonPathParser.convertToJniObject(instructions) - val insSize = jniInstructions.length - JSONUtils.getJsonObject(lhs.getBase, insSize, jniInstructions) + JSONUtils.getJsonObject(lhs.getBase, jniInstructions) } case None => GpuColumnVector.columnVectorFromNull(lhs.getRowCount.toInt, StringType) } From a1a462359fdac1a45d2bc2679b3459c8f4dde84d Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 27 Mar 2024 18:23:04 +0800 Subject: [PATCH 11/11] Adds test cases Signed-off-by: Haoyang Li --- .../src/main/python/get_json_test.py | 82 ++++++++++++++----- 1 file changed, 62 insertions(+), 20 deletions(-) diff --git a/integration_tests/src/main/python/get_json_test.py b/integration_tests/src/main/python/get_json_test.py index b0e8fcec03f..62a8710379d 100644 --- a/integration_tests/src/main/python/get_json_test.py +++ b/integration_tests/src/main/python/get_json_test.py @@ -59,8 +59,7 @@ def test_get_json_object_single_quotes(): lambda spark: spark.createDataFrame(data,schema=schema).select( f.get_json_object('jsonStr',r'''$['a']''').alias('sub_a'), f.get_json_object('jsonStr',r'''$['b']''').alias('sub_b'), - f.get_json_object('jsonStr',r'''$['c']''').alias('sub_c')), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + f.get_json_object('jsonStr',r'''$['c']''').alias('sub_c'))) @pytest.mark.parametrize('query',["$.store.bicycle", "$['store'].bicycle", @@ -100,8 +99,7 @@ def test_get_json_object_spark_unit_tests(query): ['{"big": "' + ('x' * 3000) + '"}']] assert_gpu_and_cpu_are_equal_collect( lambda spark: spark.createDataFrame(data,schema=schema).select( - f.get_json_object('jsonStr', query)), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + f.get_json_object('jsonStr', query))) # @pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/10218") def test_get_json_object_normalize_non_string_output(): @@ -122,8 +120,7 @@ def test_get_json_object_normalize_non_string_output(): assert_gpu_and_cpu_are_equal_collect( lambda spark: spark.createDataFrame(data,schema=schema).select( f.col('jsonStr'), - f.get_json_object('jsonStr', '$')), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + f.get_json_object('jsonStr', '$'))) def test_get_json_object_quoted_question(): schema = StructType([StructField("jsonStr", StringType())]) @@ -131,8 +128,7 @@ def test_get_json_object_quoted_question(): assert_gpu_and_cpu_are_equal_collect( lambda spark: spark.createDataFrame(data,schema=schema).select( - f.get_json_object('jsonStr',r'''$['?']''').alias('question')), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + f.get_json_object('jsonStr',r'''$['?']''').alias('question'))) def test_get_json_object_escaped_string_data(): schema = StructType([StructField("jsonStr", StringType())]) @@ -146,8 +142,7 @@ def test_get_json_object_escaped_string_data(): [r'{"a":"A\tB"}']] assert_gpu_and_cpu_are_equal_collect( - lambda spark: spark.createDataFrame(data,schema=schema).selectExpr('get_json_object(jsonStr,"$.a")'), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + lambda spark: spark.createDataFrame(data,schema=schema).selectExpr('get_json_object(jsonStr,"$.a")')) def test_get_json_object_escaped_key(): schema = StructType([StructField("jsonStr", StringType())]) @@ -184,8 +179,7 @@ def test_get_json_object_escaped_key(): f.get_json_object('jsonStr','$.a\n').alias('qan2'), f.get_json_object('jsonStr', r'$.a\t').alias('qat1'), f.get_json_object('jsonStr','$.a\t').alias('qat2') - ), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + )) def test_get_json_object_invalid_path(): schema = StructType([StructField("jsonStr", StringType())]) @@ -207,8 +201,7 @@ def test_get_json_object_invalid_path(): f.get_json_object('jsonStr', 'a').alias('just_a'), f.get_json_object('jsonStr', '[-1]').alias('neg_one_index'), f.get_json_object('jsonStr', '$.c[-1]').alias('c_neg_one_index'), - ), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + )) def test_get_json_object_top_level_array_notation(): # This is a special version of invalid path. It is something that the GPU supports @@ -224,8 +217,7 @@ def test_get_json_object_top_level_array_notation(): f.get_json_object('jsonStr', '$[1]').alias('one_index'), f.get_json_object('jsonStr', '''['a']''').alias('sub_a'), f.get_json_object('jsonStr', '''$['b']''').alias('sub_b'), - ), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + )) def test_get_json_object_unquoted_array_notation(): # This is a special version of invalid path. It is something that the GPU supports @@ -240,8 +232,7 @@ def test_get_json_object_unquoted_array_notation(): f.get_json_object('jsonStr', '$[a]').alias('a_index'), f.get_json_object('jsonStr', '$[1]').alias('one_index'), f.get_json_object('jsonStr', '''$['1']''').alias('quoted_one_index'), - f.get_json_object('jsonStr', '$[a1]').alias('a_one_index')), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + f.get_json_object('jsonStr', '$[a1]').alias('a_one_index'))) def test_get_json_object_white_space_removal(): @@ -278,9 +269,60 @@ def test_get_json_object_white_space_removal(): f.get_json_object('jsonStr', "$[' a. a']").alias('space_a_dot_space_a'), f.get_json_object('jsonStr', "$['a .a ']").alias('a_space_dot_a_space'), f.get_json_object('jsonStr', "$[' a . a ']").alias('space_a_space_dot_space_a_space'), - ), - conf={'spark.rapids.sql.expression.GetJsonObject': 'true'}) + )) + +def test_get_json_object_jni_java_tests(): + schema = StructType([StructField("jsonStr", StringType())]) + data = [['\'abc\''], + ['[ [11, 12], [21, [221, [2221, [22221, 22222]]]], [31, 32] ]'], + ['123'], + ['{ \'k\' : \'v\' }'], + ['[ [[[ {\'k\': \'v1\'} ], {\'k\': \'v2\'}]], [[{\'k\': \'v3\'}], {\'k\': \'v4\'}], {\'k\': \'v5\'} ]'], + ['[1, [21, 22], 3]'], + ['[ {\'k\': [0, 1, 2]}, {\'k\': [10, 11, 12]}, {\'k\': [20, 21, 22]} ]'], + ['[ [0], [10, 11, 12], [2] ]'], + ['[[0, 1, 2], [10, [111, 112, 113], 12], [20, 21, 22]]'], + ['[[0, 1, 2], [10, [], 12], [20, 21, 22]]'], + ['{\'k\' : [0,1,2]}'], + ['{\'k\' : null}'] + ] + + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.createDataFrame(data,schema=schema).select( + f.col('jsonStr'), + f.get_json_object('jsonStr', '$').alias('dollor'), + f.get_json_object('jsonStr', '$[*][*]').alias('s_w_s_w'), + f.get_json_object('jsonStr', '$.k').alias('dot_k'), + f.get_json_object('jsonStr', '$[*]').alias('s_w'), + f.get_json_object('jsonStr', '$[*].k[*]').alias('s_w_k_s_w'), + f.get_json_object('jsonStr', '$[1][*]').alias('s_1_s_w'), + f.get_json_object('jsonStr', "$[1][1][*]").alias('s_1_s_1_s_w'), + f.get_json_object('jsonStr', "$.k[1]").alias('dot_k_s_1'), + f.get_json_object('jsonStr', "$.*").alias('w'), + )) + + +@allow_non_gpu('ProjectExec') +def test_deep_nested_json(): + schema = StructType([StructField("jsonStr", StringType())]) + data = [['{"a":{"b":{"c":{"d":{"e":{"f":{"g":{"h":{"i":{"j":{"k":{"l":{"m":{"n":{"o":{"p":{"q":{"r":{"s":{"t":{"u":{"v":{"w":{"x":{"y":{"z":"A"}}' + ]] + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.createDataFrame(data,schema=schema).select( + f.get_json_object('jsonStr', '$.a.b.c.d.e.f.g.h.i').alias('i'), + f.get_json_object('jsonStr', '$.a.b.c.d.e.f.g.h.i.j.k.l.m.n.o.p').alias('p') + )) + +@allow_non_gpu('ProjectExec') +def test_deep_nested_json_fallback(): + schema = StructType([StructField("jsonStr", StringType())]) + data = [['{"a":{"b":{"c":{"d":{"e":{"f":{"g":{"h":{"i":{"j":{"k":{"l":{"m":{"n":{"o":{"p":{"q":{"r":{"s":{"t":{"u":{"v":{"w":{"x":{"y":{"z":"A"}}' + ]] + assert_gpu_fallback_collect( + lambda spark: spark.createDataFrame(data,schema=schema).select( + f.get_json_object('jsonStr', '$.a.b.c.d.e.f.g.h.i.j.k.l.m.n.o.p.q.r.s.t.u.v.w.x.y.z').alias('z')), + 'GetJsonObject') @allow_non_gpu('ProjectExec') @pytest.mark.parametrize('json_str_pattern', [r'\{"store": \{"fruit": \[\{"weight":\d,"type":"[a-z]{1,9}"\}\], ' \