From 3d3ade222e23d4878a62f3bd86992e39abaa9790 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 11 Mar 2024 09:48:10 -0500 Subject: [PATCH] Make JSON parsing common between JsonToStructs and ScanJson (#10542) Signed-off-by: Andy Grove Signed-off-by: Robert (Bobby) Evans Co-authored-by: Andy Grove --- .../advanced_configs.md | 2 +- docs/compatibility.md | 157 ++++---- docs/supported_ops.md | 4 +- .../src/main/python/json_matrix_test.py | 272 +++++++------- .../src/main/python/json_test.py | 164 +++++--- .../decimal_locale_formatted_strings.json | 13 +- ...l_locale_formatted_strings_non_aribic.json | 7 + .../src/test/resources/float_formatted.json | 5 + .../resources/float_formatted_strings.json | 5 + .../resources/invalid_ridealong_columns.json | 43 +++ .../src/test/resources/nested-structs.ndjson | 3 + .../src/test/resources/optional-fields.ndjson | 3 + .../src/test/resources/sci_formatted.json | 6 + .../test/resources/sci_formatted_strings.json | 6 + .../withBackslashEscapingAnyCharacter.json | 1 + .../resources/withNumericLeadingZeros.json | 36 +- .../com/nvidia/spark/rapids/GpuCSVScan.scala | 7 +- .../nvidia/spark/rapids/GpuOverrides.scala | 14 +- .../rapids/GpuTextBasedPartitionReader.scala | 354 +++++++++--------- .../com/nvidia/spark/rapids/RapidsConf.scala | 5 +- .../catalyst/json/rapids/GpuJsonScan.scala | 281 ++++++-------- .../json/rapids/GpuReadJsonFileFormat.scala | 5 +- .../hive/rapids/GpuHiveTableScanExec.scala | 5 +- .../spark/sql/rapids/GpuJsonReadCommon.scala | 331 ++++++++++++++++ .../spark/sql/rapids/GpuJsonToStructs.scala | 179 +-------- .../rapids/shims/GpuJsonToStructsShim.scala | 11 +- .../rapids/shims/GpuJsonToStructsShim.scala | 12 +- .../rapids/shims/GpuJsonToStructsShim.scala | 16 +- 28 files changed, 1111 insertions(+), 836 deletions(-) create mode 100644 integration_tests/src/test/resources/decimal_locale_formatted_strings_non_aribic.json create mode 100644 integration_tests/src/test/resources/invalid_ridealong_columns.json create mode 100644 integration_tests/src/test/resources/nested-structs.ndjson create mode 100644 integration_tests/src/test/resources/optional-fields.ndjson create mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala rename sql-plugin/src/main/spark311/scala/{com/nvidia/spark => org/apache/spark/sql}/rapids/shims/GpuJsonToStructsShim.scala (90%) rename sql-plugin/src/main/spark320/scala/{com/nvidia/spark => org/apache/spark/sql}/rapids/shims/GpuJsonToStructsShim.scala (86%) rename sql-plugin/src/main/spark340/scala/{com/nvidia/spark => org/apache/spark/sql}/rapids/shims/GpuJsonToStructsShim.scala (86%) diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index 83b9eb4fc7f..dacf6cf4667 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -126,7 +126,7 @@ Name | Description | Default Value | Applicable at spark.rapids.sql.join.leftOuter.enabled|When set to true left outer joins are enabled on the GPU|true|Runtime spark.rapids.sql.join.leftSemi.enabled|When set to true left semi joins are enabled on the GPU|true|Runtime spark.rapids.sql.join.rightOuter.enabled|When set to true right outer joins are enabled on the GPU|true|Runtime -spark.rapids.sql.json.read.decimal.enabled|JSON reading is not 100% compatible when reading decimals.|false|Runtime +spark.rapids.sql.json.read.decimal.enabled|When reading a quoted string as a decimal Spark supports reading non-ascii unicode digits, and the RAPIDS Accelerator does not.|true|Runtime spark.rapids.sql.json.read.double.enabled|JSON reading is not 100% compatible when reading doubles.|true|Runtime spark.rapids.sql.json.read.float.enabled|JSON reading is not 100% compatible when reading floats.|true|Runtime spark.rapids.sql.json.read.mixedTypesAsString.enabled|JSON reading is not 100% compatible when reading mixed types as string.|false|Runtime diff --git a/docs/compatibility.md b/docs/compatibility.md index adcbc9e5cc9..b482ac70ffb 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -316,89 +316,71 @@ case. ## JSON -The JSON format read is a very experimental feature which is expected to have some issues, so we disable +The JSON format read is an experimental feature which is expected to have some issues, so we disable it by default. If you would like to test it, you need to enable `spark.rapids.sql.format.json.enabled` and `spark.rapids.sql.format.json.read.enabled`. -Reading input containing invalid JSON format (in any row) will throw runtime exception. -An example of valid input is as following: -``` console -{"name":"Andy", "age":30} -{"name":"Justin", "age":19} -``` - -The following input is invalid and will cause error: -```console -{"name":"Andy", "age":30} ,,,, -{"name":"Justin", "age":19} -``` - -```console -{"name": Justin", "age":19} -``` - -Reading input with duplicated json key names is also incompatible with CPU Spark. - -### JSON supporting types - -In the current version, nested types (array, struct, and map types) are not yet supported in regular JSON parsing. - -### `from_json` function +### Invalid JSON -This particular function supports to output a map or struct type with limited functionalities. +In Apache Spark on the CPU if a line in the JSON file is invalid the entire row is considered +invalid and will result in nulls being returned for all columns. It is considered invalid if it +violates the JSON specification, but with a few extensions. -The `from_json` function is disabled by default because it is experimental and has some known incompatibilities -with Spark, and can be enabled by setting `spark.rapids.sql.expression.JsonToStructs=true`. + * Single quotes are allowed to quote strings and keys + * Unquoted values like NaN and Infinity can be parsed as floating point values + * Control characters do not need to be replaced with the corresponding escape sequences in a + quoted string. + * Garbage at the end of a row, if there is valid JSON at the beginning of the row, is ignored. -Dates are partially supported but there are some known issues: +The GPU implementation does the same kinds of validations, but many of them are done on a per-column +basis, which, for example, means if a number is formatted incorrectly, it is likely only that value +will be considered invalid and return a null instead of nulls for the entire row. -- Only the default `dateFormat` of `yyyy-MM-dd` is supported in Spark 3.1.x. The query will fall back to CPU if any other format - is specified ([#9667](https://github.com/NVIDIA/spark-rapids/issues/9667)) -- Strings containing integers with more than four digits will be - parsed as null ([#9664](https://github.com/NVIDIA/spark-rapids/issues/9664)) whereas Spark versions prior to 3.4 - will parse these numbers as number of days since the epoch, and in Spark 3.4 and later, an exception will be thrown. +There are options that can be used to enable and disable many of these features which are mostly +listed below. -Timestamps are partially supported but there are some known issues: +### JSON options -- Only the default `timestampFormat` of `yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]` is supported. The query will fall back to CPU if any other format - is specified ([#9273](https://github.com/NVIDIA/spark-rapids/issues/9723)) -- Strings containing integers with more than four digits will be - parsed as null ([#9664](https://github.com/NVIDIA/spark-rapids/issues/9664)) whereas Spark versions prior to 3.4 - will parse these numbers as number of days since the epoch, and in Spark 3.4 and later, an exception will be thrown. -- Strings containing special date constant values such as `now` and `today` will parse as null ([#9724](https://github.com/NVIDIA/spark-rapids/issues/9724)), - which differs from the behavior in Spark 3.1.x +Spark supports passing options to the JSON parser when reading a dataset. In most cases if the RAPIDS Accelerator +sees one of these options that it does not support it will fall back to the CPU. In some cases we do not. The +following options are documented below. -When reading numeric values, the GPU implementation always supports leading zeros regardless of the setting -for the JSON option `allowNumericLeadingZeros` ([#9588](https://github.com/NVIDIA/spark-rapids/issues/9588)). +- `allowNumericLeadingZeros` - Allows leading zeros in numbers (e.g. 00012). By default this is set to false. + When it is false Spark considers the JSON invalid if it encounters this type of number. The RAPIDS + Accelerator supports validating columns that are returned to the user with this option on or off. -For struct output type, the function only supports struct of struct, array, string, integral, floating-point, and -decimal types. The output is incompatible if duplicated json key names are present in the input strings. For schemas -that include IntegerType, if arbitrarily large numbers are specified in the JSON strings, the GPU implementation will -cast the numbers to IntegerType, whereas CPU Spark will return null. +- `allowUnquotedControlChars` - Allows JSON Strings to contain unquoted control characters (ASCII characters with + value less than 32, including tab and line feed characters) or not. By default this is set to false. If the schema + is provided while reading JSON file, then this flag has no impact on the RAPIDS Accelerator as it always allows + unquoted control characters but Spark sees these are invalid are returns nulls. However, if the schema is not provided + and this option is false, then RAPIDS Accelerator's behavior is same as Spark where an exception is thrown + as discussed in `JSON Schema discovery` section. -In particular, the output map is not resulted from a regular JSON parsing but instead it will just contain plain text of key-value pairs extracted directly from the input JSON string. Due to such limitations, the input JSON map type schema must be `MAP` and nothing else. Furthermore, there is no validation, no error tolerance, no data conversion as well as string formatting is performed. This may lead to some minor differences in the output if compared to the result of Spark CPU's `from_json`, such as: - * Floating point numbers in the input JSON string such as `1.2000` will not be reformatted to `1.2`. Instead, the output will be the same as the input. - * If the input JSON is given as multiple rows, any row containing invalid JSON format will be parsed as an empty - struct instead of a null value ([#9592](https://github.com/NVIDIA/spark-rapids/issues/9592)). +- `allowNonNumericNumbers` - Allows `NaN` and `Infinity` values to be parsed (note that these are not valid numeric + values in the [JSON specification](https://json.org)). Spark versions prior to 3.3.0 have inconsistent behavior and will + parse some variants of `NaN` and `Infinity` even when this option is disabled + ([SPARK-38060](https://issues.apache.org/jira/browse/SPARK-38060)). The RAPIDS Accelerator behavior is consistent with + Spark version 3.3.0 and later. -When a JSON attribute contains mixed types (different types in different rows), such as a mix of dictionaries -and lists, Spark will return a string representation of the JSON, but when running on GPU, the default -behavior is to throw an exception. There is an experimental setting -`spark.rapids.sql.json.read.mixedTypesAsString.enabled` that can be set to true to support reading -mixed types as string, but there are known issues where it could also read structs as string in some cases. There -can also be minor formatting differences. Spark will return a parsed and formatted representation, but the -GPU implementation returns the unparsed JSON string. +### Nesting +In versions of Spark before 3.5.0 there is no maximum to how deeply nested JSON can be. After +3.5.0 this was updated to be 1000 by default. The current GPU implementation limits this to 254 +no matter what version of Spark is used. If the nesting level is over this the JSON is considered +invalid and all values will be returned as nulls. -### `to_json` function +Only structs are supported for nested types. There are also some issues with arrays of structs. If +your data includes this, even if you are not reading it, you might get an exception. You can +try to set `spark.rapids.sql.json.read.mixedTypesAsString.enabled` to true to work around this, +but it also has some issues with it. -The `to_json` function is disabled by default because it is experimental and has some known incompatibilities -with Spark, and can be enabled by setting `spark.rapids.sql.expression.StructsToJson=true`. +Dates and Timestamps have some issues and may return values for technically invalid inputs. -Known issues are: +Floating point numbers have issues generally like with the rest of Spark, and we can parse them into +a valid floating point number, but it might not match 100% with the way Spark does it. -- There can be rounding differences when formatting floating-point numbers as strings. For example, Spark may - produce `-4.1243574E26` but the GPU may produce `-4.124357351E26`. -- Not all JSON options are respected +Strings are supported, but the data returned might not be normalized in the same way as the CPU +implementation. Generally this comes down to the GPU not modifying the input, whereas Spark will +do things like remove extra white space and parse numbers before turning them back into a string. ### JSON Floating Point @@ -413,9 +395,9 @@ consistent with the behavior in Spark 3.3.0 and later. Another limitation of the GPU JSON reader is that it will parse strings containing non-string boolean or numeric values where Spark will treat them as invalid inputs and will just return `null`. -### JSON Timestamps +### JSON Timestamps/Dates -The JSON parser does not support the `TimestampNTZ` type and will fall back to CPU if `spark.sql.timestampType` is +The JSON parser does not support the `TimestampNTZ` type and will fall back to CPU if `spark.sql.timestampType` is set to `TIMESTAMP_NTZ` or if an explicit schema is provided that contains the `TimestampNTZ` type. There is currently no support for reading numeric values as timestamps and null values are returned instead @@ -429,28 +411,31 @@ handles schema discovery and there is no GPU acceleration of this. By default Sp dataset to determine the schema. This means that some options/errors which are ignored by the GPU may still result in an exception if used with schema discovery. -### JSON options +### `from_json` function -Spark supports passing options to the JSON parser when reading a dataset. In most cases if the RAPIDS Accelerator -sees one of these options that it does not support it will fall back to the CPU. In some cases we do not. The -following options are documented below. +`JsonToStructs` of `from_json` is based on the same code as reading a JSON lines file. There are +a few differences with it. -- `allowNumericLeadingZeros` - Allows leading zeros in numbers (e.g. 00012). By default this is set to false. -When it is false Spark throws an exception if it encounters this type of number. The RAPIDS Accelerator -strips off leading zeros from all numbers and this config has no impact on it. +The `from_json` function is disabled by default because it is experimental and has some known +incompatibilities with Spark, and can be enabled by setting +`spark.rapids.sql.expression.JsonToStructs=true`. You don't need to set +`spark.rapids.sql.format.json.enabled` and`spark.rapids.sql.format.json.read.enabled` to true. -- `allowUnquotedControlChars` - Allows JSON Strings to contain unquoted control characters (ASCII characters with -value less than 32, including tab and line feed characters) or not. By default this is set to false. If the schema -is provided while reading JSON file, then this flag has no impact on the RAPIDS Accelerator as it always allows -unquoted control characters but Spark reads these entries incorrectly as null. However, if the schema is not provided -and when the option is false, then RAPIDS Accelerator's behavior is same as Spark where an exception is thrown -as discussed in `JSON Schema discovery` section. +There is no schema discovery as a schema is required as input to `from_json` -- `allowNonNumericNumbers` - Allows `NaN` and `Infinity` values to be parsed (note that these are not valid numeric -values in the [JSON specification](https://json.org)). Spark versions prior to 3.3.0 have inconsistent behavior and will -parse some variants of `NaN` and `Infinity` even when this option is disabled -([SPARK-38060](https://issues.apache.org/jira/browse/SPARK-38060)). The RAPIDS Accelerator behavior is consistent with -Spark version 3.3.0 and later. +In addition to `structs`, a top level `map` type is supported, but only if the key and value are +strings. + +### `to_json` function + +The `to_json` function is disabled by default because it is experimental and has some known incompatibilities +with Spark, and can be enabled by setting `spark.rapids.sql.expression.StructsToJson=true`. + +Known issues are: + +- There can be rounding differences when formatting floating-point numbers as strings. For example, Spark may + produce `-4.1243574E26` but the GPU may produce `-4.124357351E26`. +- Not all JSON options are respected ### get_json_object diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 70f24e1e437..65873a40a5a 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -20379,9 +20379,9 @@ dates or timestamps, or for a lack of type coercion support. NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, MAP, UDT
NS -NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, MAP, UDT
NS diff --git a/integration_tests/src/main/python/json_matrix_test.py b/integration_tests/src/main/python/json_matrix_test.py index 98cb62286c6..823d8e0fe0e 100644 --- a/integration_tests/src/main/python/json_matrix_test.py +++ b/integration_tests/src/main/python/json_matrix_test.py @@ -30,7 +30,7 @@ def read_impl(spark): reader = reader.schema(schema) for key, value in options.items(): reader = reader.option(key, value) - return debug_df(reader.json(data_path)) + return reader.json(data_path) return read_impl def read_json_sql(data_path, schema, spark_tmp_table_factory, options = {}): @@ -56,7 +56,10 @@ def read_json_as_text(spark, data_path, column_name): } _enable_json_to_structs_conf = { - 'spark.rapids.sql.expression.JsonToStructs': 'true' + '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' } _enable_get_json_object_conf = { @@ -259,7 +262,6 @@ def test_from_json_allow_numeric_leading_zeros_on(std_input_path): # Off is the default so it really needs to work @approximate_float() -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/9588') @pytest.mark.parametrize('read_func', [read_json_df, read_json_sql]) def test_scan_json_allow_numeric_leading_zeros_off(std_input_path, read_func, spark_tmp_table_factory): assert_gpu_and_cpu_are_equal_collect( @@ -271,10 +273,10 @@ def test_scan_json_allow_numeric_leading_zeros_off(std_input_path, read_func, sp # Off is the default so it really needs to work @approximate_float() -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/9588') @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_from_json_allow_numeric_leading_zeros_off(std_input_path): schema = WITH_NUMERIC_LEAD_ZEROS_SCHEMA + assert_gpu_and_cpu_are_equal_collect( lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NUMERIC_LEAD_ZEROS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowNumericLeadingZeros": "false"})), conf =_enable_json_to_structs_conf) @@ -303,7 +305,7 @@ def test_json_tuple_allow_numeric_leading_zeros_off(std_input_path): @approximate_float() @pytest.mark.parametrize('read_func', [read_json_df, read_json_sql]) -@pytest.mark.xfail(condition = is_before_spark_330(), reason = 'https://github.com/NVIDIA/spark-rapids/issues/10493') +@pytest.mark.xfail(condition=is_before_spark_330(), reason='https://issues.apache.org/jira/browse/SPARK-38060') def test_scan_json_allow_nonnumeric_numbers_off(std_input_path, read_func, spark_tmp_table_factory): assert_gpu_and_cpu_are_equal_collect( read_func(std_input_path + '/' + WITH_NONNUMERIC_NUMBERS_FILE, @@ -314,7 +316,6 @@ def test_scan_json_allow_nonnumeric_numbers_off(std_input_path, read_func, spark @approximate_float() @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10456') @pytest.mark.xfail(condition = is_before_spark_330(), reason = 'https://github.com/NVIDIA/spark-rapids/issues/10493') def test_from_json_allow_nonnumeric_numbers_off(std_input_path): schema = WITH_NONNUMERIC_NUMBERS_SCHEMA @@ -338,7 +339,6 @@ def test_scan_json_allow_nonnumeric_numbers_on(std_input_path, read_func, spark_ @approximate_float() @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 @pytest.mark.xfail(condition = is_before_spark_330(), reason = 'https://github.com/NVIDIA/spark-rapids/issues/10493') -@pytest.mark.xfail(condition = is_spark_330_or_later(), reason = 'https://github.com/NVIDIA/spark-rapids/issues/10494') def test_from_json_allow_nonnumeric_numbers_on(std_input_path): schema = WITH_NONNUMERIC_NUMBERS_SCHEMA assert_gpu_and_cpu_are_equal_collect( @@ -475,29 +475,47 @@ def test_json_tuple_allow_unquoted_control_chars_on(std_input_path): WITH_DEC_LOCALE_FILE = "decimal_locale_formatted_strings.json" +WITH_DEC_LOCALE_NON_ARIBIC_FILE = "decimal_locale_formatted_strings_non_aribic.json" WITH_DEC_LOCALE_SCHEMA = StructType([ StructField("data", DecimalType(10, 5))]) -DEC_LOCALES=["en-US","it-CH","ko-KR","h-TH-x-lvariant-TH","ru-RU","de-DE","iw-IL","hi-IN","ar-QA","zh-CN","ko-KR"] +NON_US_DEC_LOCALES=["it-CH","ko-KR","h-TH-x-lvariant-TH","ru-RU","de-DE","iw-IL","hi-IN","ar-QA","zh-CN","ko-KR"] -# Off is the default for scan so it really needs to work +# US is the default locale so we kind of what it to work @pytest.mark.parametrize('read_func', [read_json_df]) # We don't need both they are always the same -@pytest.mark.parametrize('locale', DEC_LOCALES) -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10470') -def test_scan_json_dec_locale(std_input_path, read_func, spark_tmp_table_factory,locale): +def test_scan_json_dec_locale_US(std_input_path, read_func, spark_tmp_table_factory): assert_gpu_and_cpu_are_equal_collect( + read_func(std_input_path + '/' + WITH_DEC_LOCALE_FILE, + WITH_DEC_LOCALE_SCHEMA, + spark_tmp_table_factory), + conf=_enable_all_types_json_scan_conf) + +# We don't support the other locales yet, so we fall back to the CPU +@allow_non_gpu('FileSourceScanExec') +@pytest.mark.parametrize('read_func', [read_json_df]) # We don't need both they are always the same +@pytest.mark.parametrize('locale', NON_US_DEC_LOCALES) +def test_scan_json_dec_locale(std_input_path, read_func, spark_tmp_table_factory, locale): + assert_gpu_fallback_collect( read_func(std_input_path + '/' + WITH_DEC_LOCALE_FILE, WITH_DEC_LOCALE_SCHEMA, spark_tmp_table_factory, - {"locale": "locale"}), + {"locale": locale}), + 'FileSourceScanExec', conf=_enable_all_types_json_scan_conf) @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 -@pytest.mark.parametrize('locale', DEC_LOCALES) -@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10470') -def test_from_json_dec_locale(std_input_path, locale): +def test_from_json_dec_locale_US(std_input_path): schema = WITH_DEC_LOCALE_SCHEMA assert_gpu_and_cpu_are_equal_collect( + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema)), + conf =_enable_json_to_structs_conf) + +@allow_non_gpu(TEXT_INPUT_EXEC, 'ProjectExec', *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 +@pytest.mark.parametrize('locale', NON_US_DEC_LOCALES) +def test_from_json_dec_locale(std_input_path, locale): + schema = WITH_DEC_LOCALE_SCHEMA + assert_gpu_fallback_collect( lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"locale": locale})), + 'JsonToStructs', conf =_enable_json_to_structs_conf) #There is no way to set a locale for these, and it really should not matter @@ -515,16 +533,77 @@ def test_json_tuple_dec_locale(std_input_path): conf =_enable_json_tuple_conf) -@pytest.mark.parametrize('input_file', [ +#################################################################### +# Spark supports non-aribic numbers, but they should be really rare +#################################################################### + +# US is the default locale so we kind of what it to work +@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10532') +@pytest.mark.parametrize('read_func', [read_json_df]) # We don't need both they are always the same +def test_scan_json_dec_locale_US_non_aribic(std_input_path, read_func, spark_tmp_table_factory): + assert_gpu_and_cpu_are_equal_collect( + read_func(std_input_path + '/' + WITH_DEC_LOCALE_NON_ARIBIC_FILE, + WITH_DEC_LOCALE_SCHEMA, + spark_tmp_table_factory), + conf=_enable_all_types_json_scan_conf) + +# We don't support the other locales yet, so we fall back to the CPU +@allow_non_gpu('FileSourceScanExec') +@pytest.mark.parametrize('read_func', [read_json_df]) # We don't need both they are always the same +@pytest.mark.parametrize('locale', NON_US_DEC_LOCALES) +def test_scan_json_dec_locale_non_aribic(std_input_path, read_func, spark_tmp_table_factory, locale): + assert_gpu_fallback_collect( + read_func(std_input_path + '/' + WITH_DEC_LOCALE_NON_ARIBIC_FILE, + WITH_DEC_LOCALE_SCHEMA, + spark_tmp_table_factory, + {"locale": locale}), + 'FileSourceScanExec', + conf=_enable_all_types_json_scan_conf) + +@pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10532') +@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 +def test_from_json_dec_locale_US_non_aribic(std_input_path): + schema = WITH_DEC_LOCALE_SCHEMA + assert_gpu_and_cpu_are_equal_collect( + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_NON_ARIBIC_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema)), + conf =_enable_json_to_structs_conf) + +@allow_non_gpu(TEXT_INPUT_EXEC, 'ProjectExec', *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 +@pytest.mark.parametrize('locale', NON_US_DEC_LOCALES) +def test_from_json_dec_locale_non_aribic(std_input_path, locale): + schema = WITH_DEC_LOCALE_SCHEMA + assert_gpu_fallback_collect( + lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_NON_ARIBIC_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"locale": locale})), + 'JsonToStructs', + conf =_enable_json_to_structs_conf) + +#There is no way to set a locale for these, and it really should not matter +@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) + +#There is no way to set a locale for these, and it really should not matter +@allow_non_gpu(TEXT_INPUT_EXEC) +def test_json_tuple_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('''json_tuple(json, "data")'''), + conf =_enable_json_tuple_conf) + +# These are common files used by most of the tests. A few files are for specific types, but these are very targeted tests +COMMON_TEST_FILES=[ "int_formatted.json", "float_formatted.json", "sci_formatted.json", - pytest.param("int_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), + "int_formatted_strings.json", "float_formatted_strings.json", "sci_formatted_strings.json", "decimal_locale_formatted_strings.json", "single_quoted_strings.json", - "boolean_formatted.json"]) + "boolean_formatted.json"] + +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @pytest.mark.parametrize('read_func', [read_json_df]) # we have done so many tests already that we don't need both read func. They are the same def test_scan_json_bytes(std_input_path, read_func, spark_tmp_table_factory, input_file): assert_gpu_and_cpu_are_equal_collect( @@ -533,16 +612,7 @@ def test_scan_json_bytes(std_input_path, read_func, spark_tmp_table_factory, inp spark_tmp_table_factory), conf=_enable_all_types_json_scan_conf) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - pytest.param("float_formatted.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10460')), - pytest.param("sci_formatted.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10460')), - "int_formatted_strings.json", - "float_formatted_strings.json", - "sci_formatted_strings.json", - "decimal_locale_formatted_strings.json", - "single_quoted_strings.json", - "boolean_formatted.json"]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_from_json_bytes(std_input_path, input_file): schema = StructType([StructField("data", ByteType())]) @@ -550,16 +620,7 @@ def test_from_json_bytes(std_input_path, input_file): lambda spark : read_json_as_text(spark, std_input_path + '/' + input_file, "json").select(f.col('json'), f.from_json(f.col('json'), schema)), conf =_enable_json_to_structs_conf) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - "float_formatted.json", - "sci_formatted.json", - pytest.param("int_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - "float_formatted_strings.json", - "sci_formatted_strings.json", - pytest.param("decimal_locale_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - "single_quoted_strings.json", - "boolean_formatted.json"]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @pytest.mark.parametrize('read_func', [read_json_df]) # we have done so many tests already that we don't need both read func. They are the same def test_scan_json_shorts(std_input_path, read_func, spark_tmp_table_factory, input_file): assert_gpu_and_cpu_are_equal_collect( @@ -568,16 +629,7 @@ def test_scan_json_shorts(std_input_path, read_func, spark_tmp_table_factory, in spark_tmp_table_factory), conf=_enable_all_types_json_scan_conf) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - pytest.param("float_formatted.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10460')), - pytest.param("sci_formatted.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10460')), - "int_formatted_strings.json", - "float_formatted_strings.json", - "sci_formatted_strings.json", - "decimal_locale_formatted_strings.json", - "single_quoted_strings.json", - "boolean_formatted.json"]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_from_json_shorts(std_input_path, input_file): schema = StructType([StructField("data", ShortType())]) @@ -585,16 +637,7 @@ def test_from_json_shorts(std_input_path, input_file): lambda spark : read_json_as_text(spark, std_input_path + '/' + input_file, "json").select(f.col('json'), f.from_json(f.col('json'), schema)), conf =_enable_json_to_structs_conf) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - "float_formatted.json", - "sci_formatted.json", - pytest.param("int_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - "float_formatted_strings.json", - "sci_formatted_strings.json", - pytest.param("decimal_locale_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - "single_quoted_strings.json", - "boolean_formatted.json"]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @pytest.mark.parametrize('read_func', [read_json_df]) # we have done so many tests already that we don't need both read func. They are the same def test_scan_json_ints(std_input_path, read_func, spark_tmp_table_factory, input_file): assert_gpu_and_cpu_are_equal_collect( @@ -603,16 +646,7 @@ def test_scan_json_ints(std_input_path, read_func, spark_tmp_table_factory, inpu spark_tmp_table_factory), conf=_enable_all_types_json_scan_conf) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - pytest.param("float_formatted.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10460')), - pytest.param("sci_formatted.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10460')), - "int_formatted_strings.json", - "float_formatted_strings.json", - "sci_formatted_strings.json", - "decimal_locale_formatted_strings.json", - "single_quoted_strings.json", - "boolean_formatted.json"]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_from_json_ints(std_input_path, input_file): schema = StructType([StructField("data", IntegerType())]) @@ -620,16 +654,7 @@ def test_from_json_ints(std_input_path, input_file): lambda spark : read_json_as_text(spark, std_input_path + '/' + input_file, "json").select(f.col('json'), f.from_json(f.col('json'), schema)), conf =_enable_json_to_structs_conf) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - "float_formatted.json", - "sci_formatted.json", - pytest.param("int_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - "float_formatted_strings.json", - "sci_formatted_strings.json", - pytest.param("decimal_locale_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - "single_quoted_strings.json", - "boolean_formatted.json"]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @pytest.mark.parametrize('read_func', [read_json_df]) # we have done so many tests already that we don't need both read func. They are the same def test_scan_json_longs(std_input_path, read_func, spark_tmp_table_factory, input_file): assert_gpu_and_cpu_are_equal_collect( @@ -638,16 +663,7 @@ def test_scan_json_longs(std_input_path, read_func, spark_tmp_table_factory, inp spark_tmp_table_factory), conf=_enable_all_types_json_scan_conf) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - pytest.param("float_formatted.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10460')), - pytest.param("sci_formatted.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10460')), - "int_formatted_strings.json", - "float_formatted_strings.json", - "sci_formatted_strings.json", - "decimal_locale_formatted_strings.json", - "single_quoted_strings.json", - "boolean_formatted.json"]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_from_json_longs(std_input_path, input_file): schema = StructType([StructField("data", LongType())]) @@ -656,16 +672,7 @@ def test_from_json_longs(std_input_path, input_file): conf =_enable_json_to_structs_conf) @pytest.mark.parametrize('dt', [DecimalType(38,0), DecimalType(38,10), DecimalType(10,2)], ids=idfn) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - "float_formatted.json", - "sci_formatted.json", - "int_formatted_strings.json", - pytest.param("float_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10469')), - "sci_formatted_strings.json", - pytest.param("decimal_locale_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10470')), - "single_quoted_strings.json", - "boolean_formatted.json"]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @pytest.mark.parametrize('read_func', [read_json_df]) # we have done so many tests already that we don't need both read func. They are the same def test_scan_json_decs(std_input_path, read_func, spark_tmp_table_factory, input_file, dt): assert_gpu_and_cpu_are_equal_collect( @@ -675,16 +682,7 @@ def test_scan_json_decs(std_input_path, read_func, spark_tmp_table_factory, inpu conf=_enable_all_types_json_scan_conf) @pytest.mark.parametrize('dt', [DecimalType(38,0), DecimalType(38,10), DecimalType(10,2)], ids=idfn) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - pytest.param("float_formatted.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10467')), - "sci_formatted.json", - "int_formatted_strings.json", - "float_formatted_strings.json", - "sci_formatted_strings.json", - pytest.param("decimal_locale_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10470')), - "single_quoted_strings.json", - "boolean_formatted.json"]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_from_json_decs(std_input_path, input_file, dt): schema = StructType([StructField("data", dt)]) @@ -701,8 +699,9 @@ def test_from_json_decs(std_input_path, input_file, dt): "float_formatted_strings.json", "sci_formatted_strings.json", "decimal_locale_formatted_strings.json", - pytest.param("single_quoted_strings.json", marks=pytest.mark.xfail(condition=is_before_spark_330(),reason='https://github.com/NVIDIA/spark-rapids/issues/10495')), - pytest.param("boolean_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10479'))]) + pytest.param("single_quoted_strings.json", marks=pytest.mark.xfail(condition=is_before_spark_330(), reason='https://github.com/NVIDIA/spark-rapids/issues/10495')), + pytest.param("boolean_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10479')), + pytest.param("invalid_ridealong_columns.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10534'))]) @pytest.mark.parametrize('read_func', [read_json_df]) def test_scan_json_strings(std_input_path, read_func, spark_tmp_table_factory, input_file): assert_gpu_and_cpu_are_equal_collect( @@ -720,7 +719,8 @@ def test_scan_json_strings(std_input_path, read_func, spark_tmp_table_factory, i "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/10479'))]) + pytest.param("boolean_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10479')), + pytest.param("invalid_ridealong_columns.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10534'))]) @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_from_json_strings(std_input_path, input_file): schema = StructType([StructField("data", StringType())]) @@ -737,7 +737,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("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'))]) @allow_non_gpu(TEXT_INPUT_EXEC) def test_get_json_object_formats(std_input_path, input_file): assert_gpu_and_cpu_are_equal_collect( @@ -753,23 +754,15 @@ def test_get_json_object_formats(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("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'))]) @allow_non_gpu(TEXT_INPUT_EXEC) def test_json_tuple_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("*", '''json_tuple(json, "data")'''), conf =_enable_json_tuple_conf) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - "float_formatted.json", - "sci_formatted.json", - "int_formatted_strings.json", - "float_formatted_strings.json", - "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/10480'))]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @pytest.mark.parametrize('read_func', [read_json_df]) def test_scan_json_bools(std_input_path, read_func, spark_tmp_table_factory, input_file): assert_gpu_and_cpu_are_equal_collect( @@ -778,16 +771,7 @@ def test_scan_json_bools(std_input_path, read_func, spark_tmp_table_factory, inp spark_tmp_table_factory), conf=_enable_all_types_json_scan_conf) -@pytest.mark.parametrize('input_file', [ - "int_formatted.json", - "float_formatted.json", - pytest.param("sci_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10480')), - "int_formatted_strings.json", - "float_formatted_strings.json", - "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/10480'))]) +@pytest.mark.parametrize('input_file', COMMON_TEST_FILES) @allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_from_json_bools(std_input_path, input_file): schema = StructType([StructField("data", BooleanType())]) @@ -800,10 +784,10 @@ def test_from_json_bools(std_input_path, input_file): "int_formatted.json", pytest.param("float_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10481')), "sci_formatted.json", - pytest.param("int_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - pytest.param("float_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - pytest.param("sci_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - pytest.param("decimal_locale_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), + "int_formatted_strings.json", + pytest.param("float_formatted_strings.json", marks=pytest.mark.xfail(condition=is_before_spark_330(), reason='https://issues.apache.org/jira/browse/SPARK-38060')), + "sci_formatted_strings.json", + "decimal_locale_formatted_strings.json", "single_quoted_strings.json", "boolean_formatted.json"]) @pytest.mark.parametrize('read_func', [read_json_df]) @@ -820,7 +804,7 @@ def test_scan_json_floats(std_input_path, read_func, spark_tmp_table_factory, in pytest.param("float_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10481')), "sci_formatted.json", "int_formatted_strings.json", - pytest.param("float_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), + pytest.param("float_formatted_strings.json", marks=pytest.mark.xfail(condition=is_before_spark_330(), reason='https://issues.apache.org/jira/browse/SPARK-38060')), "sci_formatted_strings.json", "decimal_locale_formatted_strings.json", "single_quoted_strings.json", @@ -837,10 +821,10 @@ def test_from_json_floats(std_input_path, input_file): "int_formatted.json", pytest.param("float_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10481')), "sci_formatted.json", - pytest.param("int_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - pytest.param("float_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - pytest.param("sci_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), - pytest.param("decimal_locale_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), + "int_formatted_strings.json", + pytest.param("float_formatted_strings.json", marks=pytest.mark.xfail(condition=is_before_spark_330(), reason='https://issues.apache.org/jira/browse/SPARK-38060')), + "sci_formatted_strings.json", + "decimal_locale_formatted_strings.json", "single_quoted_strings.json", "boolean_formatted.json"]) @pytest.mark.parametrize('read_func', [read_json_df]) @@ -857,7 +841,7 @@ def test_scan_json_doubles(std_input_path, read_func, spark_tmp_table_factory, i pytest.param("float_formatted.json", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10481')), "sci_formatted.json", "int_formatted_strings.json", - pytest.param("float_formatted_strings.json",marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10468')), + pytest.param("float_formatted_strings.json", marks=pytest.mark.xfail(condition=is_before_spark_330(), reason='https://issues.apache.org/jira/browse/SPARK-38060')), "sci_formatted_strings.json", "decimal_locale_formatted_strings.json", "single_quoted_strings.json", diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 9a8d7a8ac13..890eb43b4a8 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -40,6 +40,7 @@ optional_whitespace_regex = '[ \t\xA0\u1680\u180e\u2000-\u200a\u202f\u205f\u3000]?' _enable_all_types_conf = { + 'spark.rapids.sql.expression.JsonToStructs': 'true', 'spark.rapids.sql.format.json.enabled': 'true', 'spark.rapids.sql.format.json.read.enabled': 'true', 'spark.rapids.sql.json.read.float.enabled': 'true', @@ -320,12 +321,12 @@ def do_read(spark): ]) @pytest.mark.parametrize('schema', [_bool_schema, _byte_schema, _short_schema, _int_schema, _long_schema, \ _float_schema, _double_schema, _decimal_10_2_schema, _decimal_10_3_schema, \ - _date_schema]) + _date_schema], ids=idfn) @pytest.mark.parametrize('read_func', [read_json_df, read_json_sql]) @pytest.mark.parametrize('allow_non_numeric_numbers', ['true', 'false']) @pytest.mark.parametrize('allow_numeric_leading_zeros', [ 'true', - pytest.param('false', marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/9588')), + 'false' ]) @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) @allow_non_gpu(*not_utc_allow_for_test_json_scan) @@ -377,7 +378,7 @@ def test_read_invalid_json(spark_tmp_table_factory, std_input_path, read_func, f 'mixed-primitive-arrays.ndjson', ]) @pytest.mark.parametrize('read_func', [read_json_df, read_json_sql]) -@pytest.mark.parametrize('schema', [_int_schema]) +@pytest.mark.parametrize('schema', [_int_schema], ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "json"]) def test_read_valid_json(spark_tmp_table_factory, std_input_path, read_func, filename, schema, v1_enabled_list): conf = copy_and_update(_enable_all_types_conf, @@ -390,6 +391,53 @@ def test_read_valid_json(spark_tmp_table_factory, std_input_path, read_func, fil {}), conf=conf) +@pytest.mark.parametrize('filename', ['nested-structs.ndjson']) +@pytest.mark.parametrize('schema', [ + StructType([StructField('teacher', StringType())]), + StructType([ + StructField('student', StructType([ + StructField('name', StringType()), + StructField('age', IntegerType()) + ])) + ]), + StructType([ + StructField('teacher', StringType()), + StructField('student', StructType([ + StructField('name', StringType()), + StructField('age', IntegerType()) + ])) + ]), +]) +@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql]) +@pytest.mark.parametrize('v1_enabled_list', ["", "json"]) +def test_read_nested_struct(spark_tmp_table_factory, std_input_path, read_func, filename, schema, v1_enabled_list): + conf = copy_and_update(_enable_all_types_conf, {'spark.sql.sources.useV1SourceList': v1_enabled_list}) + assert_gpu_and_cpu_are_equal_collect( + read_func(std_input_path + '/' + filename, + schema, + spark_tmp_table_factory, + {}), + conf=conf) + +@pytest.mark.parametrize('filename', ['optional-fields.ndjson']) +@pytest.mark.parametrize('schema', [ + StructType([StructField('teacher', StringType())]), + StructType([StructField('student', StringType())]), + StructType([ + StructField('teacher', StringType()), + StructField('student', StringType()) + ]), +]) +@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql]) +@pytest.mark.parametrize('v1_enabled_list', ["", "json"]) +def test_read_optional_fields(spark_tmp_table_factory, std_input_path, read_func, filename, schema, v1_enabled_list): + conf = copy_and_update(_enable_all_types_conf, {'spark.sql.sources.useV1SourceList': v1_enabled_list}) + assert_gpu_and_cpu_are_equal_collect( + read_func(std_input_path + '/' + filename, + schema, + spark_tmp_table_factory, + {}), + conf=conf) # allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' not_utc_json_scan_allow=['FileSourceScanExec'] if is_not_utc() else [] @@ -569,7 +617,7 @@ def test_from_json_map(): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.from_json(f.col('a'), 'MAP')), - conf={"spark.rapids.sql.expression.JsonToStructs": True}) + conf=_enable_all_types_conf) @allow_non_gpu('ProjectExec', 'JsonToStructs') def test_from_json_map_fallback(): @@ -580,7 +628,7 @@ def test_from_json_map_fallback(): lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.from_json(f.col('a'), 'MAP')), 'JsonToStructs', - conf={"spark.rapids.sql.expression.JsonToStructs": True}) + conf=_enable_all_types_conf) @pytest.mark.parametrize('schema', [ 'struct', @@ -597,21 +645,21 @@ def test_from_json_map_fallback(): ]) @allow_non_gpu(*non_utc_allow) def test_from_json_struct(schema): - # note that column 'a' does not use leading zeroes due to https://github.com/NVIDIA/spark-rapids/issues/9588 + # note that column 'a' does not use leading zeroes due to https://github.com/NVIDIA/spark-rapids/issues/10534 json_string_gen = StringGen(r'{\'a\': [1-9]{0,5}, "b": \'[A-Z]{0,5}\', "c": 1\d\d\d}') \ .with_special_pattern('', weight=50) \ .with_special_pattern('null', weight=50) assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.col('a'), f.from_json('a', schema)), - conf={"spark.rapids.sql.expression.JsonToStructs": True}) + conf=_enable_all_types_conf) @pytest.mark.parametrize('schema', [ 'struct', ]) @allow_non_gpu("ProjectExec") def test_from_json_struct_fallback_dupe_keys(schema): - # note that column 'a' does not use leading zeroes due to https://github.com/NVIDIA/spark-rapids/issues/9588 + # note that column 'a' does not use leading zeroes due to https://github.com/NVIDIA/spark-rapids/issues/10534 json_string_gen = StringGen(r'{\'a\': [1-9]{0,5}, "b": \'[A-Z]{0,5}\', "c": 1\d\d\d}') \ .with_special_pattern('', weight=50) \ .with_special_pattern('null', weight=50) @@ -619,7 +667,7 @@ def test_from_json_struct_fallback_dupe_keys(schema): lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.col('a'), f.from_json('a', schema)), 'JsonToStructs', - conf={"spark.rapids.sql.expression.JsonToStructs": True}) + conf=_enable_all_types_conf) @pytest.mark.parametrize('pattern', [ r'{ "bool": (true|false|True|False|TRUE|FALSE) }', @@ -638,7 +686,7 @@ def test_from_json_struct_boolean(pattern): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.col('a'), f.from_json('a', 'struct')), - conf={"spark.rapids.sql.expression.JsonToStructs": True}) + conf=_enable_all_types_conf) @allow_non_gpu(*non_utc_allow) @datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/10349') @@ -648,8 +696,8 @@ def test_from_json_struct_decimal(): .with_special_pattern('null', weight=50) assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ - .select(f.from_json('a', 'struct')), - conf={"spark.rapids.sql.expression.JsonToStructs": True}) + .select(f.col('a'), f.from_json('a', 'struct')), + conf=_enable_all_types_conf) @pytest.mark.parametrize('date_gen', [ # "yyyy-MM-dd" @@ -673,16 +721,17 @@ def test_from_json_struct_decimal(): ]) @pytest.mark.parametrize('date_format', [None, 'yyyy-MM-dd'] if is_before_spark_320 else json_supported_date_formats) @allow_non_gpu(*non_utc_project_allow) +@pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10535') def test_from_json_struct_date(date_gen, date_format): json_string_gen = StringGen(r'{ "a": ' + date_gen + ' }') \ .with_special_case('{ "a": null }') \ .with_special_case('null') options = { 'dateFormat': date_format } if date_format else { } + conf = copy_and_update(_enable_all_types_conf, {'spark.sql.legacy.timeParserPolicy': 'CORRECTED'}) assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.col('a'), f.from_json('a', 'struct', options)), - conf={'spark.rapids.sql.expression.JsonToStructs': True, - 'spark.sql.legacy.timeParserPolicy': 'CORRECTED'}) + conf=conf) @allow_non_gpu('ProjectExec') @pytest.mark.parametrize('date_gen', ["\"[1-8]{1}[0-9]{3}-[0-3]{1,2}-[0-3]{1,2}\""]) @@ -695,12 +744,12 @@ def test_from_json_struct_date_fallback_legacy(date_gen, date_format): .with_special_case('{ "a": null }') \ .with_special_case('null') options = { 'dateFormat': date_format } if date_format else { } + conf = copy_and_update(_enable_all_types_conf, {'spark.sql.legacy.timeParserPolicy': 'LEGACY'}) assert_gpu_fallback_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.col('a'), f.from_json('a', 'struct', options)), 'ProjectExec', - conf={"spark.rapids.sql.expression.JsonToStructs": True, - 'spark.sql.legacy.timeParserPolicy': 'LEGACY'}) + conf=conf) @pytest.mark.skipif(is_spark_320_or_later(), reason="We only fallback for non-default formats prior to 320") @allow_non_gpu('ProjectExec') @@ -714,12 +763,12 @@ def test_from_json_struct_date_fallback_non_default_format(date_gen, date_format .with_special_case('{ "a": null }') \ .with_special_case('null') options = { 'dateFormat': date_format } + conf = copy_and_update(_enable_all_types_conf, {'spark.sql.legacy.timeParserPolicy': 'CORRECTED'}) assert_gpu_fallback_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.col('a'), f.from_json('a', 'struct', options)), 'ProjectExec', - conf={"spark.rapids.sql.expression.JsonToStructs": True, - 'spark.sql.legacy.timeParserPolicy': 'CORRECTED'}) + conf=conf) # allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' non_utc_project_allow = ['ProjectExec'] if is_not_utc() else [] @@ -740,7 +789,7 @@ def test_from_json_struct_date_fallback_non_default_format(date_gen, date_format # "nnnnn" (number of days since epoch prior to Spark 3.4, throws exception from 3.4) pytest.param("\"" + optional_whitespace_regex + "[0-9]{5}" + optional_whitespace_regex + "\"", marks=pytest.mark.skip(reason="https://github.com/NVIDIA/spark-rapids/issues/9664")), # integral - pytest.param("[0-9]{1,5}", marks=pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/9588")), + pytest.param("[0-9]{1,5}", marks=pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/4940")), pytest.param("[1-9]{1,8}", marks=pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/4940")), # floating-point r"[0-9]{0,2}\.[0-9]{1,2}" @@ -757,7 +806,7 @@ def test_from_json_struct_date_fallback_non_default_format(date_gen, date_format ]) @pytest.mark.parametrize('time_parser_policy', [ pytest.param("LEGACY", marks=pytest.mark.allow_non_gpu('ProjectExec')), - "CORRECTED" + pytest.param("CORRECTED", marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10535')) ]) @pytest.mark.parametrize('ansi_enabled', [ True, False ]) def test_from_json_struct_timestamp(timestamp_gen, timestamp_format, time_parser_policy, ansi_enabled): @@ -766,12 +815,13 @@ def test_from_json_struct_timestamp(timestamp_gen, timestamp_format, time_parser .with_special_case('{ "a": "6395-12-21T56:86:40.205705Z" }') \ .with_special_case('null') options = { 'timestampFormat': timestamp_format } if timestamp_format else { } + conf = copy_and_update(_enable_all_types_conf, { + 'spark.sql.legacy.timeParserPolicy': time_parser_policy, + 'spark.sql.ansi.enabled': ansi_enabled}) assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.col('a'), f.from_json('a', 'struct', options)), - conf={"spark.rapids.sql.expression.JsonToStructs": True, - 'spark.sql.legacy.timeParserPolicy': time_parser_policy, - 'spark.sql.ansi.enabled': ansi_enabled }) + conf=conf) @allow_non_gpu('ProjectExec') @pytest.mark.parametrize('timestamp_gen', ["\"[1-8]{1}[0-9]{3}-[0-3]{1,2}-[0-3]{1,2}T[0-9]{1,2}:[0-9]{1,2}:[0-9]{1,2}(\\.[0-9]{1,6})?Z?\""]) @@ -784,12 +834,12 @@ def test_from_json_struct_timestamp_fallback_legacy(timestamp_gen, timestamp_for .with_special_case('{ "a": null }') \ .with_special_case('null') options = { 'timestampFormat': timestamp_format } if timestamp_format else { } + conf = copy_and_update(_enable_all_types_conf, {'spark.sql.legacy.timeParserPolicy': 'LEGACY'}) assert_gpu_fallback_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.col('a'), f.from_json('a', 'struct', options)), 'ProjectExec', - conf={"spark.rapids.sql.expression.JsonToStructs": True, - 'spark.sql.legacy.timeParserPolicy': 'LEGACY'}) + conf=conf) @allow_non_gpu('ProjectExec') @pytest.mark.parametrize('timestamp_gen', ["\"[1-8]{1}[0-9]{3}-[0-3]{1,2}-[0-3]{1,2}T[0-9]{1,2}:[0-9]{1,2}:[0-9]{1,2}(\\.[0-9]{1,6})?Z?\""]) @@ -797,21 +847,24 @@ def test_from_json_struct_timestamp_fallback_legacy(timestamp_gen, timestamp_for "yyyy-MM-dd'T'HH:mm:ss.SSSXXX", "dd/MM/yyyy'T'HH:mm:ss[.SSS][XXX]", ]) +@pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10535') def test_from_json_struct_timestamp_fallback_non_default_format(timestamp_gen, timestamp_format): json_string_gen = StringGen(r'{ "a": ' + timestamp_gen + ' }') \ .with_special_case('{ "a": null }') \ .with_special_case('null') options = { 'timestampFormat': timestamp_format } if timestamp_format else { } + conf = copy_and_update(_enable_all_types_conf, {'spark.sql.legacy.timeParserPolicy': 'CORRECTED'}) assert_gpu_fallback_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.col('a'), f.from_json('a', 'struct', options)), 'ProjectExec', - conf={"spark.rapids.sql.expression.JsonToStructs": True, - 'spark.sql.legacy.timeParserPolicy': 'CORRECTED'}) + conf=conf) -@pytest.mark.parametrize('schema', ['struct', - 'struct>', - 'struct>']) +@pytest.mark.parametrize('schema', [ + 'struct', + 'struct>', + 'struct>' +]) @allow_non_gpu(*non_utc_allow) def test_from_json_struct_of_struct(schema): json_string_gen = StringGen(r'{"teacher": "[A-Z]{1}[a-z]{2,5}",' \ @@ -822,7 +875,7 @@ def test_from_json_struct_of_struct(schema): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.from_json('a', schema)), - conf={"spark.rapids.sql.expression.JsonToStructs": True}) + conf=_enable_all_types_conf) @pytest.mark.parametrize('schema', ['struct', 'struct>>', @@ -837,7 +890,7 @@ def test_from_json_struct_of_list(schema): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.from_json('a', schema)), - conf={"spark.rapids.sql.expression.JsonToStructs": True}) + conf=_enable_all_types_conf) @pytest.mark.parametrize('schema', [ 'struct' @@ -846,11 +899,11 @@ def test_from_json_struct_of_list(schema): @pytest.mark.xfail(reason = 'https://github.com/NVIDIA/spark-rapids/issues/10351') def test_from_json_mixed_types_list_struct(schema): json_string_gen = StringGen(r'{"a": (\[1,2,3\]|{"b":"[a-z]{2}"}) }') + conf = copy_and_update(_enable_all_types_conf, {'spark.rapids.sql.json.read.mixedTypesAsString.enabled': 'true'}) assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select('a', f.from_json('a', schema)), - conf={"spark.rapids.sql.expression.JsonToStructs": True, - 'spark.rapids.sql.json.read.mixedTypesAsString.enabled': True}) + conf=conf) @pytest.mark.parametrize('schema', ['struct', 'struct']) @allow_non_gpu(*non_utc_allow) @@ -859,7 +912,7 @@ def test_from_json_struct_all_empty_string_input(schema): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, json_string_gen) \ .select(f.from_json('a', schema)), - conf={"spark.rapids.sql.expression.JsonToStructs": True}) + conf=_enable_all_types_conf) @allow_non_gpu('FileSourceScanExec') @pytest.mark.skipif(is_before_spark_340(), reason='enableDateTimeParsingFallback is supported from Spark3.4.0') @@ -1117,7 +1170,7 @@ def test_spark_from_json_escaping(): [r"""{'"quote':20}"""]] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # from_json # from_json null input column @@ -1128,7 +1181,7 @@ def test_spark_from_json(): [None]] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # from_json - input=empty array, schema=struct, output=single row with null # from_json - input=empty object, schema=struct, output=single row with null @@ -1143,7 +1196,7 @@ def test_spark_from_json_empty_table(data): schema = StructType([StructField("a", IntegerType())]) assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # SPARK-20549: from_json bad UTF-8 @pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10483') @@ -1153,7 +1206,7 @@ def test_spark_from_json_bad_json(): data = [["\u0000\u0000\u0000A\u0001AAA"]] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # from_json - invalid data @allow_non_gpu(*non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 @@ -1163,7 +1216,7 @@ def test_spark_from_json_invalid(): [None]] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # This does not work the same way as the unit test. We fallback, and nulls are allowed as input, so we will just go with it for now # If we ever do try to support FAILFAST this shold be updated so that there is an invalid JSON line and we produce the proper @@ -1176,7 +1229,7 @@ def test_spark_from_json_invalid_failfast(): assert_gpu_fallback_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema, {'mode': 'FAILFAST'})), 'ProjectExec', - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) #from_json - input=array, schema=array, output=array #from_json - input=object, schema=array, output=array of single row @@ -1192,7 +1245,7 @@ def test_spark_from_json_array_schema(): assert_gpu_fallback_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), 'ProjectExec', - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # from_json - input=array of single object, schema=struct, output=single row @allow_non_gpu(*non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 @@ -1201,7 +1254,7 @@ def test_spark_from_json_single_item_array_to_struct(): data = [[r'''[{"a": 1}]''']] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) @pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10484') #from_json - input=array, schema=struct, output=single row @@ -1212,7 +1265,7 @@ def test_spark_from_json_struct_with_corrupted_row(): assert_gpu_fallback_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema, {'columnNameOfCorruptRecord': 'corrupted'})), 'ProjectExec', - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # The Spark test sets the time zone to several in a list, but we are just going to go off of our TZ testing for selecting time zones # for this part of the test @@ -1223,7 +1276,7 @@ def test_spark_from_json_timestamp_default_format(): data = [[r'''{"t": "2016-01-01T00:00:00.123Z"}''']] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # The spark test only sets the timezone as an ID one, but we really just care that we fallback when it is set to something we cannot support @pytest.mark.parametrize('zone_id', [ @@ -1236,6 +1289,7 @@ def test_spark_from_json_timestamp_default_format(): "Asia/Hong_Kong", "Europe/Brussels"], ids=idfn) @allow_non_gpu('ProjectExec') +@pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10535') # This is expected to fallback to the CPU because the timestampFormat is not supported, but really is, so we shold be better about this. def test_spark_from_json_timestamp_format_option_zoneid(zone_id): schema = StructType([StructField("t", TimestampType())]) @@ -1243,7 +1297,7 @@ def test_spark_from_json_timestamp_format_option_zoneid(zone_id): assert_gpu_fallback_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema, {'timestampFormat': "yyyy-MM-dd'T'HH:mm:ss",'timeZone': zone_id})), 'ProjectExec', - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) @pytest.mark.parametrize('zone_id', [ "UTC", @@ -1261,7 +1315,7 @@ def test_spark_from_json_timestamp_format_option_zoneid_but_supported_format(zon data = [[r'''{"t": "2016-01-01 00:00:00"}''']] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema, {'timestampFormat': "yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]",'timeZone': zone_id})), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) @pytest.mark.parametrize('zone_id', [ "UTC", @@ -1279,7 +1333,7 @@ def test_spark_from_json_timestamp_format_option_zoneid_but_default_format(zone_ [r'''{"t": "2023-07-27 12:21:05"}''']] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema, {'timeZone': zone_id})), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # from_json with option (timestampFormat) # no timestamp format appears to actually work @@ -1290,7 +1344,7 @@ def test_spark_from_json_timestamp_format(): assert_gpu_fallback_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema, {'timestampFormat': "dd/MM/yyyy HH:mm"})), 'ProjectExec', - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # from_json missing fields @pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10489') @@ -1303,7 +1357,7 @@ def test_spark_from_json_missing_fields_with_cr(): }"""]] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) @allow_non_gpu(*non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_spark_from_json_missing_fields(): @@ -1311,7 +1365,7 @@ def test_spark_from_json_missing_fields(): data = [["""{"a": 1,"c": "foo"}"""]] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # For now we are going to try and rely on the dateFormat to fallback, but we might want to # fallback for unsupported locals too @@ -1326,7 +1380,7 @@ def test_spark_from_json_date_with_locale(data, locale): assert_gpu_fallback_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema, {'dateFormat': 'MMM yyyy', 'locale': locale})), 'ProjectExec', - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) @allow_non_gpu(*non_utc_allow) @pytest.mark.skipif(is_before_spark_320(), reason="dd/MM/yyyy is supported in 3.2.0 and after") @@ -1336,7 +1390,7 @@ def test_spark_from_json_date_with_format(): schema = StructType([StructField("d", DateType())]) assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema, {'dateFormat': 'dd/MM/yyyy'})), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # TEST from_json missing columns @allow_non_gpu(*non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 @@ -1345,7 +1399,7 @@ def test_spark_from_json_missing_columns(): data = [['''{"a": 1}''']] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) # TEST from_json invalid json @pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10483') @@ -1355,4 +1409,4 @@ def test_spark_from_json_invalid_json(): data = [['''{"a" 1}''']] assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.createDataFrame(data, 'json STRING').select(f.col('json'), f.from_json(f.col('json'), schema)), - conf = { 'spark.rapids.sql.expression.JsonToStructs': True }) + conf =_enable_all_types_conf) diff --git a/integration_tests/src/test/resources/decimal_locale_formatted_strings.json b/integration_tests/src/test/resources/decimal_locale_formatted_strings.json index 56fa2f37aa7..c645e66c7f7 100644 --- a/integration_tests/src/test/resources/decimal_locale_formatted_strings.json +++ b/integration_tests/src/test/resources/decimal_locale_formatted_strings.json @@ -1,13 +1,14 @@ -{"data":"१,०००.००१"} {"data":"1,000.001"} {"data":"1.000,001"} {"data":"1 000,001"} {"data":"1'000.001"} -{"data":"๑,๐๐๐.๐๐๑"} {"data":"1000.001"} {"data":"1000"} {"data":"BAD"} -{"data": "१"} -{"data": "๑"} -{"data": "१०"} -{"data": "๑๐"} +{"data": "1,2,3,4"} +{"data": "1,2,3.4"} +{"data": "0123.4"} +{"data": "0,1234"} +{"data": " 1234"} +{"data": "1234 "} +{"data": "1 234"} diff --git a/integration_tests/src/test/resources/decimal_locale_formatted_strings_non_aribic.json b/integration_tests/src/test/resources/decimal_locale_formatted_strings_non_aribic.json new file mode 100644 index 00000000000..f49d8b22d83 --- /dev/null +++ b/integration_tests/src/test/resources/decimal_locale_formatted_strings_non_aribic.json @@ -0,0 +1,7 @@ +{"data":"१,०००.००१"} +{"data":"๑,๐๐๐.๐๐๑"} +{"data": "१"} +{"data": "๑"} +{"data": "१०"} +{"data": "๑๐"} +{"data": "1٢७"} diff --git a/integration_tests/src/test/resources/float_formatted.json b/integration_tests/src/test/resources/float_formatted.json index e52f6249b2b..8f305c3dbed 100644 --- a/integration_tests/src/test/resources/float_formatted.json +++ b/integration_tests/src/test/resources/float_formatted.json @@ -18,3 +18,8 @@ {"data": 0.99} {"data": 0.999} {"data": 0.9999} +{"data": +1.0} +{"data": -1.0} +{"data": 1 . 0} +{"data": - 1 . 0} +{"data": + 1 . 0} diff --git a/integration_tests/src/test/resources/float_formatted_strings.json b/integration_tests/src/test/resources/float_formatted_strings.json index 38dd5dc54ce..1f8047e12e6 100644 --- a/integration_tests/src/test/resources/float_formatted_strings.json +++ b/integration_tests/src/test/resources/float_formatted_strings.json @@ -13,3 +13,8 @@ {"data": "0.99"} {"data": "0.999"} {"data": "0.9999"} +{"data": "+1.0"} +{"data": "-1.0"} +{"data": "1 . 0"} +{"data": "- 1 . 0"} +{"data": "+ 1 . 0"} diff --git a/integration_tests/src/test/resources/invalid_ridealong_columns.json b/integration_tests/src/test/resources/invalid_ridealong_columns.json new file mode 100644 index 00000000000..e45013747d5 --- /dev/null +++ b/integration_tests/src/test/resources/invalid_ridealong_columns.json @@ -0,0 +1,43 @@ +{"ride-along-bool": TRUE, "data": "BAD BOOL TRUE"} +{"data": "BAD BOOL FALSE", "ride-along-bool": FALSE} +{"data": "BAD BOOL T", "ride-along-bool": T} +{"data": "BAD BOOL F", "ride-along-bool": F} +{"data": "BAD BOOL t", "ride-along-bool": t} +{"data": "BAD BOOL f", "ride-along-bool": f} +{"data": "GOOD BOOL true", "ride-along-bool": true} +{"data": "GOOD BOOL false", "ride-along-bool": false} +{"data": "BAD NUM 1.", "ride-along-num": 1.} +{"data": "BAD NUM 1f", "ride-along-num": 1f} +{"data": "BAD NUM .7f", "ride-along-num": .7f} +{"data": "BAD NUM 0xA", "ride-along-num": 0xA} +{"data": "BAD NUM -.1", "ride-along-num": -.1} +{"data": "BAD NUM +1", "ride-along-num": +1} +{"data": "BAD NUM 01", "ride-along-num": 01} +{"data": "BAD NUM 00.1", "ride-along-num": 00.1} +{"data": "BAD NUM 1 000", "ride-along-num": 1 000} +{"data": "BAD NUM 1,000", "ride-along-num": 1,000} +{"data": "BAD NUM 1e", "ride-along-num": 1e} +{"data": "BAD NUM 1ee2", "ride-along-num": 1ee2} +{"data": "BAD NUM 1ex2", "ride-along-num": 1ex2} +{"data": "BAD NUM NaN", "ride-along-num": NaN} +{"data": "BAD NUM nan", "ride-along-num": nan} +{"data": "BAD NUM INF", "ride-along-num": INF} +{"data": "BAD NUM +INF", "ride-along-num": +INF} +{"data": "BAD NUM -INF", "ride-along-num": -INF} +{"data": "BAD NUM -Infinity", "ride-along-num": -Infinity} +{"data": "BAD NUM Infinity", "ride-along-num": Infinity} +{"data": "BAD NUM +Infinity", "ride-along-num": +Infinity} +{"data": "GOOD NUM 1", "ride-along-num": 1} +{"data": "GOOD NUM 1e10", "ride-along-num": 1e10} +{"data": "GOOD NUM 1E10", "ride-along-num": 1E10} +{"data": "GOOD NUM 1e+10", "ride-along-num": 1e+10} +{"data": "GOOD NUM 1E-10", "ride-along-num": 1E-10} +{"data": "GOOD NUM 0.1", "ride-along-num": 0.1} +{"data": "BAD STRING BACKSLASH V", "ride-along-str": "TEST \V"} +{"data": "BAD STRING BACKSLASH z", "ride-along-str": "TEST \z"} +{"data": "GOOD STRING TAB", "ride-along-str": "TEST AFTER"} +{"data": "GOOD STRING TEST", "ride-along-str": "TEST"} +{"data": "GOOD null", "ride-along-bool": null} +{"data": "BAD NULL", "ride-along-bool": NULL} +{"data": "BAD empty (1)", "ride-along-bool": ,"other": "TEST"} +{"data": "BAD empty (2)", "ride-along-bool": } diff --git a/integration_tests/src/test/resources/nested-structs.ndjson b/integration_tests/src/test/resources/nested-structs.ndjson new file mode 100644 index 00000000000..e3cf2fec68a --- /dev/null +++ b/integration_tests/src/test/resources/nested-structs.ndjson @@ -0,0 +1,3 @@ +{ "teacher": "Bob" } +{ "student": { "name": "Carol", "age": 21 } } +{ "teacher": "Bob", "student": { "name": "Carol", "age": 21 } } diff --git a/integration_tests/src/test/resources/optional-fields.ndjson b/integration_tests/src/test/resources/optional-fields.ndjson new file mode 100644 index 00000000000..4afc3e5c58d --- /dev/null +++ b/integration_tests/src/test/resources/optional-fields.ndjson @@ -0,0 +1,3 @@ +{ "teacher": "Bob" } +{ "student": "Carol" } +{ "teacher": "Bob", "student": "Carol" } diff --git a/integration_tests/src/test/resources/sci_formatted.json b/integration_tests/src/test/resources/sci_formatted.json index 97a5670c08c..2cc39c84308 100644 --- a/integration_tests/src/test/resources/sci_formatted.json +++ b/integration_tests/src/test/resources/sci_formatted.json @@ -8,3 +8,9 @@ {"data": -3.4028235E38} {"data": 1E309} {"data": 1E-309} +{"data": +1E0} +{"data": +1E1} +{"data": 1E-1} +{"data": 1E+1} +{"data": 1e1} +{"data": 1 E 1} diff --git a/integration_tests/src/test/resources/sci_formatted_strings.json b/integration_tests/src/test/resources/sci_formatted_strings.json index f4abc44c773..52b6c7e84c7 100644 --- a/integration_tests/src/test/resources/sci_formatted_strings.json +++ b/integration_tests/src/test/resources/sci_formatted_strings.json @@ -8,3 +8,9 @@ {"data": "-3.4028235E38"} {"data": "1E309"} {"data": "1E-309"} +{"data": "+1E0"} +{"data": "+1E1"} +{"data": "1E-1"} +{"data": "1E+1"} +{"data": "1e1"} +{"data": "1 E 1"} diff --git a/integration_tests/src/test/resources/withBackslashEscapingAnyCharacter.json b/integration_tests/src/test/resources/withBackslashEscapingAnyCharacter.json index 3a04bc1dabe..a37323f671d 100644 --- a/integration_tests/src/test/resources/withBackslashEscapingAnyCharacter.json +++ b/integration_tests/src/test/resources/withBackslashEscapingAnyCharacter.json @@ -1,4 +1,5 @@ {"str": "value"} +{"str": "value \u0035"} {"\str": "value"} {"st\r": "value"} {"str\": "value"} diff --git a/integration_tests/src/test/resources/withNumericLeadingZeros.json b/integration_tests/src/test/resources/withNumericLeadingZeros.json index 8468fb933f3..93d7a71c63d 100644 --- a/integration_tests/src/test/resources/withNumericLeadingZeros.json +++ b/integration_tests/src/test/resources/withNumericLeadingZeros.json @@ -1,4 +1,32 @@ -{"byte": 01, "int": 01, "float": 01.0, "decimal": 01.0} -{"byte": 001, "int": 001, "float": 001.01, "decimal": 001.01} -{"byte": -01, "int": -01, "float": -01.0, "decimal": -01.0} -{"byte": 1, "int": 1, "float": 1, "decimal": 1} +{"byte": 01} +{"byte": 001} +{"byte": -01} +{"byte": 01234} +{"byte": 0123} +{"byte": 1} +{"int": 01} +{"int": 001} +{"int": -01} +{"int": 01234} +{"int": 0123} +{"int": 1} +{"float": 01.0} +{"float": 001.01} +{"float": -01.0} +{"float": 01234} +{"float": 0123.4} +{"float": 01234} +{"float": 0123.4} +{"float": 1} +{"decimal": 01.0} +{"decimal": 001.01} +{"decimal": -01.0} +{"decimal": 01234} +{"decimal": 0123.4} +{"decimal": 1} +{"decimal": "01.0"} +{"decimal": "001.01"} +{"decimal": "-01.0"} +{"decimal": "01234"} +{"decimal": "0123.4"} +{"decimal": "1"} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala index 51d695904a3..0c4ef92ce63 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala @@ -419,20 +419,21 @@ class CSVPartitionReader( * Read the host buffer to GPU table * * @param dataBufferer buffered data to be parsed - * @param cudfSchema the cudf schema of the data + * @param cudfDataSchema the cudf schema of the data * @param readDataSchema the Spark schema describing what will be read * @param isFirstChunk if it is the first chunk * @return table */ override def readToTable( dataBufferer: HostLineBufferer, - cudfSchema: Schema, + cudfDataSchema: Schema, readDataSchema: StructType, + cudfReadDataSchema: Schema, isFirstChunk: Boolean, decodeTime: GpuMetric): Table = { val hasHeader = isFirstChunk && parsedOptions.headerFlag val csvOpts = buildCsvOptions(parsedOptions, readDataSchema, hasHeader) - CSVPartitionReader.readToTable(dataBufferer, cudfSchema, decodeTime, csvOpts, + CSVPartitionReader.readToTable(dataBufferer, cudfDataSchema, decodeTime, csvOpts, getFileFormatShortName, partFile) } } 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 d8bc93617f8..38562dfdb2f 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 @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.rapids.TimeStamp import org.apache.spark.sql.catalyst.json.rapids.GpuJsonScan +import org.apache.spark.sql.catalyst.json.rapids.GpuJsonScan.JsonToStructsReaderType import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -856,6 +857,10 @@ object GpuOverrides extends Logging { .map(r => r.wrap(expr, conf, parent, r).asInstanceOf[BaseExprMeta[INPUT]]) .getOrElse(new RuleNotFoundExprMeta(expr, conf, parent)) + val jsonStructReadTypes: TypeSig = (TypeSig.STRUCT + TypeSig.ARRAY + + TypeSig.STRING + TypeSig.integral + TypeSig.fp + TypeSig.DECIMAL_128 + TypeSig.BOOLEAN + + TypeSig.DATE + TypeSig.TIMESTAMP).nested() + lazy val fileFormats: Map[FileFormatType, Map[FileFormatOp, FileFormatChecks]] = Map( (CsvFormatType, FileFormatChecks( cudfRead = TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + @@ -894,7 +899,7 @@ object GpuOverrides extends Logging { sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.UDT).nested())), (JsonFormatType, FileFormatChecks( - cudfRead = TypeSig.commonCudfTypes + TypeSig.DECIMAL_128, + cudfRead = jsonStructReadTypes, cudfWrite = TypeSig.none, sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.UDT).nested())), @@ -3652,9 +3657,7 @@ object GpuOverrides extends Logging { expr[JsonToStructs]( "Returns a struct value with the given `jsonStr` and `schema`", ExprChecks.projectOnly( - TypeSig.STRUCT.nested(TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.STRING + TypeSig.integral + - TypeSig.fp + TypeSig.DECIMAL_64 + TypeSig.DECIMAL_128 + TypeSig.BOOLEAN + TypeSig.DATE + - TypeSig.TIMESTAMP) + + TypeSig.STRUCT.nested(jsonStructReadTypes) + TypeSig.MAP.nested(TypeSig.STRING).withPsNote(TypeEnum.MAP, "MAP only supports keys and values that are of STRING type"), (TypeSig.STRUCT + TypeSig.MAP + TypeSig.ARRAY).nested(TypeSig.all), @@ -3681,7 +3684,8 @@ object GpuOverrides extends Logging { willNotWorkOnGpu("from_json on GPU only supports MapType " + "or StructType schema") } - GpuJsonScan.tagJsonToStructsSupport(a.options, a.dataType, this) + GpuJsonScan.tagSupport(SQLConf.get, JsonToStructsReaderType, a.dataType, a.dataType, + a.options, this) } override def convertToGpu(child: Expression): GpuExpression = diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala index 2a7bff24c50..c12b8c4d5d6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala @@ -189,6 +189,125 @@ class HostStringColBufferer(size: Long, separator: Array[Byte]) extends LineBuff } } +object GpuTextBasedPartitionReader { + def castStringToTimestamp( + lhs: ColumnVector, + sparkFormat: String, + dtype: DType): ColumnVector = { + + val optionalSeconds = raw"(?:\:\d{2})?" + val optionalMicros = raw"(?:\.\d{1,6})?" + val twoDigits = raw"\d{2}" + val fourDigits = raw"\d{4}" + + val regexRoot = sparkFormat + .replace("'T'", "T") + .replace("yyyy", fourDigits) + .replace("MM", twoDigits) + .replace("dd", twoDigits) + .replace("HH", twoDigits) + .replace("mm", twoDigits) + .replace("[:ss]", optionalSeconds) + .replace(":ss", optionalSeconds) // Spark always treats seconds portion as optional + .replace("[.SSSXXX]", optionalMicros) + .replace("[.SSS][XXX]", optionalMicros) + .replace("[.SSS]", optionalMicros) + .replace("[.SSSSSS]", optionalMicros) + .replace(".SSSXXX", optionalMicros) + .replace(".SSSSSS", optionalMicros) + .replace(".SSS", optionalMicros) + + // Spark treats timestamp portion as optional always + val regexOptionalTime = regexRoot.split('T') match { + case Array(d, t) => + d + "(?:[ T]" + t + ")?" + case _ => + regexRoot + } + val regex = regexOptionalTime + raw"Z?\Z" + + // get a list of all possible cuDF formats that we need to check for + val cudfFormats = GpuTextBasedDateUtils.toCudfFormats(sparkFormat, parseString = true) + + + // filter by regexp first to eliminate invalid entries + val regexpFiltered = withResource(lhs.strip()) { stripped => + val prog = new RegexProgram(regex, CaptureGroups.NON_CAPTURE) + withResource(stripped.matchesRe(prog)) { matchesRe => + withResource(Scalar.fromNull(DType.STRING)) { nullString => + matchesRe.ifElse(stripped, nullString) + } + } + } + + // fix timestamps that have milliseconds but no microseconds + // example ".296" => ".296000" + val sanitized = withResource(regexpFiltered) { _ => + // cannot replace with back-refs directly because cuDF cannot support "\1000\2" so we + // first substitute with a placeholder and then replace that. The placeholder value + // `@` was chosen somewhat arbitrarily but should be safe since we do not support any + // date/time formats that contain the `@` character + val placeholder = "@" + val prog = new RegexProgram(raw"(\.\d{3})(Z?)\Z") + withResource(regexpFiltered.stringReplaceWithBackrefs(prog, raw"\1$placeholder\2")) { tmp => + withResource(Scalar.fromString(placeholder)) { from => + withResource(Scalar.fromString("000")) { to => + tmp.stringReplace(from, to) + } + } + } + } + + def isTimestamp(fmt: String): ColumnVector = { + val pos = fmt.indexOf('T') + if (pos == -1) { + sanitized.isTimestamp(fmt) + } else { + // Spark supports both ` ` and `T` as the delimiter so we have to test + // for both formats when calling `isTimestamp` in cuDF but the + // `asTimestamp` method ignores the delimiter so we only need to call that + // with one format + val withSpaceDelim = fmt.substring(0, pos) + ' ' + fmt.substring(pos + 1) + withResource(sanitized.isTimestamp(fmt)) { isValidFmt1 => + withResource(sanitized.isTimestamp(withSpaceDelim)) { isValidFmt2 => + isValidFmt1.or(isValidFmt2) + } + } + } + } + + def asTimestampOrNull(fmt: String): ColumnVector = { + withResource(Scalar.fromNull(dtype)) { nullScalar => + withResource(isTimestamp(fmt)) { isValid => + withResource(sanitized.asTimestamp(dtype, fmt)) { ts => + isValid.ifElse(ts, nullScalar) + } + } + } + } + + def asTimestampOr(fmt: String, orValue: ColumnVector): ColumnVector = { + withResource(orValue) { _ => + withResource(isTimestamp(fmt)) { isValid => + withResource(sanitized.asTimestamp(dtype, fmt)) { ts => + isValid.ifElse(ts, orValue) + } + } + } + } + + withResource(sanitized) { _ => + if (cudfFormats.length == 1) { + asTimestampOrNull(cudfFormats.head) + } else { + cudfFormats.tail.foldLeft(asTimestampOrNull(cudfFormats.head)) { (input, fmt) => + asTimestampOr(fmt, input) + } + } + } + } +} + /** * The text based PartitionReader * @param conf the Hadoop configuration @@ -285,6 +404,64 @@ abstract class GpuTextBasedPartitionReader[BUFF <: LineBufferer, FACT <: LineBuf } } + def getCudfSchema(dataSchema: StructType): Schema = { + // read boolean and numeric columns as strings in cuDF + val dataSchemaWithStrings = StructType(dataSchema.fields + .map(f => { + f.dataType match { + case DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | + DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | + DataTypes.DoubleType | _: DecimalType | DataTypes.DateType | + DataTypes.TimestampType => + f.copy(dataType = DataTypes.StringType) + case other if GpuTypeShims.supportCsvRead(other) => + f.copy(dataType = DataTypes.StringType) + case _ => + f + } + })) + GpuColumnVector.from(dataSchemaWithStrings) + } + + def castTableToDesiredTypes(table: Table, readSchema: StructType): Table = { + val columns = new ListBuffer[ColumnVector]() + // Table increases the ref counts on the columns so we have + // to close them after creating the table + withResource(columns) { _ => + for (i <- 0 until table.getNumberOfColumns) { + val castColumn = readSchema.fields(i).dataType match { + case DataTypes.BooleanType => + castStringToBool(table.getColumn(i)) + case DataTypes.ByteType => + castStringToInt(table.getColumn(i), DType.INT8) + case DataTypes.ShortType => + castStringToInt(table.getColumn(i), DType.INT16) + case DataTypes.IntegerType => + castStringToInt(table.getColumn(i), DType.INT32) + case DataTypes.LongType => + castStringToInt(table.getColumn(i), DType.INT64) + case DataTypes.FloatType => + castStringToFloat(table.getColumn(i), DType.FLOAT32) + case DataTypes.DoubleType => + castStringToFloat(table.getColumn(i), DType.FLOAT64) + case dt: DecimalType => + castStringToDecimal(table.getColumn(i), dt) + case DataTypes.DateType => + castStringToDate(table.getColumn(i), DType.TIMESTAMP_DAYS) + case DataTypes.TimestampType => + castStringToTimestamp(table.getColumn(i), + timestampFormat, DType.TIMESTAMP_MICROSECONDS) + case other if GpuTypeShims.supportCsvRead(other) => + GpuTypeShims.csvRead(table.getColumn(i), other) + case _ => + table.getColumn(i).incRefCount() + } + columns += castColumn + } + new Table(columns.toSeq: _*) + } + } + private def readToTable(isFirstChunk: Boolean): Option[Table] = { val (dataBuffer, dataSize) = metrics(BUFFER_TIME).ns { readPartFile() @@ -301,68 +478,19 @@ abstract class GpuTextBasedPartitionReader[BUFF <: LineBufferer, FACT <: LineBuf readDataSchema } - // read boolean and numeric columns as strings in cuDF - val dataSchemaWithStrings = StructType(dataSchema.fields - .map(f => { - f.dataType match { - case DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | - DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | - DataTypes.DoubleType | _: DecimalType | DataTypes.DateType | - DataTypes.TimestampType => - f.copy(dataType = DataTypes.StringType) - case other if GpuTypeShims.supportCsvRead(other) => - f.copy(dataType = DataTypes.StringType) - case _ => - f - } - })) - val cudfSchema = GpuColumnVector.from(dataSchemaWithStrings) + val cudfSchema = getCudfSchema(dataSchema) + val cudfReadSchema = getCudfSchema(newReadDataSchema) // about to start using the GPU GpuSemaphore.acquireIfNecessary(TaskContext.get()) // The buffer that is sent down - val table = readToTable(dataBuffer, cudfSchema, newReadDataSchema, isFirstChunk, - metrics(GPU_DECODE_TIME)) + val table = readToTable(dataBuffer, cudfSchema, newReadDataSchema, cudfReadSchema, + isFirstChunk, metrics(GPU_DECODE_TIME)) // parse boolean and numeric columns that were read as strings val castTable = withResource(table) { _ => - val columns = new ListBuffer[ColumnVector]() - // Table increases the ref counts on the columns so we have - // to close them after creating the table - withResource(columns) { _ => - for (i <- 0 until table.getNumberOfColumns) { - val castColumn = newReadDataSchema.fields(i).dataType match { - case DataTypes.BooleanType => - castStringToBool(table.getColumn(i)) - case DataTypes.ByteType => - castStringToInt(table.getColumn(i), DType.INT8) - case DataTypes.ShortType => - castStringToInt(table.getColumn(i), DType.INT16) - case DataTypes.IntegerType => - castStringToInt(table.getColumn(i), DType.INT32) - case DataTypes.LongType => - castStringToInt(table.getColumn(i), DType.INT64) - case DataTypes.FloatType => - castStringToFloat(table.getColumn(i), DType.FLOAT32) - case DataTypes.DoubleType => - castStringToFloat(table.getColumn(i), DType.FLOAT64) - case dt: DecimalType => - castStringToDecimal(table.getColumn(i), dt) - case DataTypes.DateType => - castStringToDate(table.getColumn(i), DType.TIMESTAMP_DAYS) - case DataTypes.TimestampType => - castStringToTimestamp(table.getColumn(i), timestampFormat, - DType.TIMESTAMP_MICROSECONDS) - case other if GpuTypeShims.supportCsvRead(other) => - GpuTypeShims.csvRead(table.getColumn(i), other) - case _ => - table.getColumn(i).incRefCount() - } - columns += castColumn - } - new Table(columns.toSeq: _*) - } + castTableToDesiredTypes(table, newReadDataSchema) } handleResult(newReadDataSchema, castTable) @@ -399,117 +527,7 @@ abstract class GpuTextBasedPartitionReader[BUFF <: LineBufferer, FACT <: LineBuf lhs: ColumnVector, sparkFormat: String, dtype: DType): ColumnVector = { - - val optionalSeconds = raw"(?:\:\d{2})?" - val optionalMicros = raw"(?:\.\d{1,6})?" - val twoDigits = raw"\d{2}" - val fourDigits = raw"\d{4}" - - val regexRoot = sparkFormat - .replace("'T'", "T") - .replace("yyyy", fourDigits) - .replace("MM", twoDigits) - .replace("dd", twoDigits) - .replace("HH", twoDigits) - .replace("mm", twoDigits) - .replace("[:ss]", optionalSeconds) - .replace(":ss", optionalSeconds) // Spark always treats seconds portion as optional - .replace("[.SSSXXX]", optionalMicros) - .replace("[.SSS][XXX]", optionalMicros) - .replace("[.SSS]", optionalMicros) - .replace("[.SSSSSS]", optionalMicros) - .replace(".SSSXXX", optionalMicros) - .replace(".SSSSSS", optionalMicros) - .replace(".SSS", optionalMicros) - - // Spark treats timestamp portion as optional always - val regexOptionalTime = regexRoot.split('T') match { - case Array(d, t) => - d + "(?:[ T]" + t + ")?" - case _ => - regexRoot - } - val regex = regexOptionalTime + raw"Z?\Z" - - // get a list of all possible cuDF formats that we need to check for - val cudfFormats = GpuTextBasedDateUtils.toCudfFormats(sparkFormat, parseString = true) - - - // filter by regexp first to eliminate invalid entries - val regexpFiltered = withResource(lhs.strip()) { stripped => - val prog = new RegexProgram(regex, CaptureGroups.NON_CAPTURE) - withResource(stripped.matchesRe(prog)) { matchesRe => - withResource(Scalar.fromNull(DType.STRING)) { nullString => - matchesRe.ifElse(stripped, nullString) - } - } - } - - // fix timestamps that have milliseconds but no microseconds - // example ".296" => ".296000" - val sanitized = withResource(regexpFiltered) { _ => - // cannot replace with back-refs directly because cuDF cannot support "\1000\2" so we - // first substitute with a placeholder and then replace that. The placeholder value - // `@` was chosen somewhat arbitrarily but should be safe since we do not support any - // date/time formats that contain the `@` character - val placeholder = "@" - val prog = new RegexProgram(raw"(\.\d{3})(Z?)\Z") - withResource(regexpFiltered.stringReplaceWithBackrefs(prog, raw"\1$placeholder\2")) { tmp => - withResource(Scalar.fromString(placeholder)) { from => - withResource(Scalar.fromString("000")) { to => - tmp.stringReplace(from, to) - } - } - } - } - - def isTimestamp(fmt: String): ColumnVector = { - val pos = fmt.indexOf('T') - if (pos == -1) { - sanitized.isTimestamp(fmt) - } else { - // Spark supports both ` ` and `T` as the delimiter so we have to test - // for both formats when calling `isTimestamp` in cuDF but the - // `asTimestamp` method ignores the delimiter so we only need to call that - // with one format - val withSpaceDelim = fmt.substring(0, pos) + ' ' + fmt.substring(pos + 1) - withResource(sanitized.isTimestamp(fmt)) { isValidFmt1 => - withResource(sanitized.isTimestamp(withSpaceDelim)) { isValidFmt2 => - isValidFmt1.or(isValidFmt2) - } - } - } - } - - def asTimestampOrNull(fmt: String): ColumnVector = { - withResource(Scalar.fromNull(dtype)) { nullScalar => - withResource(isTimestamp(fmt)) { isValid => - withResource(sanitized.asTimestamp(dtype, fmt)) { ts => - isValid.ifElse(ts, nullScalar) - } - } - } - } - - def asTimestampOr(fmt: String, orValue: ColumnVector): ColumnVector = { - withResource(orValue) { _ => - withResource(isTimestamp(fmt)) { isValid => - withResource(sanitized.asTimestamp(dtype, fmt)) { ts => - isValid.ifElse(ts, orValue) - } - } - } - } - - withResource(sanitized) { _ => - if (cudfFormats.length == 1) { - asTimestampOrNull(cudfFormats.head) - } else { - cudfFormats.tail.foldLeft(asTimestampOrNull(cudfFormats.head)) { (input, fmt) => - asTimestampOr(fmt, input) - } - } - } + GpuTextBasedPartitionReader.castStringToTimestamp(lhs, sparkFormat, dtype) } def castStringToBool(input: ColumnVector): ColumnVector @@ -535,15 +553,17 @@ abstract class GpuTextBasedPartitionReader[BUFF <: LineBufferer, FACT <: LineBuf /** * Read the host buffer to GPU table * @param dataBuffer where the data is buffered - * @param cudfSchema the cudf schema of the data + * @param cudfDataSchema the cudf schema of the data * @param readDataSchema the Spark schema describing what will be read + * @param cudfReadDataSchema the cudf schema of just the data we want to read. * @param isFirstChunk if it is the first chunk * @return table */ def readToTable( dataBuffer: BUFF, - cudfSchema: Schema, + cudfDataSchema: Schema, readDataSchema: StructType, + cudfReadDataSchema: Schema, isFirstChunk: Boolean, decodeTime: GpuMetric): Table diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index f52c3b5f334..9966bc9f0fc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1217,9 +1217,10 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern") .createWithDefault(true) val ENABLE_READ_JSON_DECIMALS = conf("spark.rapids.sql.json.read.decimal.enabled") - .doc("JSON reading is not 100% compatible when reading decimals.") + .doc("When reading a quoted string as a decimal Spark supports reading non-ascii " + + "unicode digits, and the RAPIDS Accelerator does not.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val ENABLE_READ_JSON_MIXED_TYPES_AS_STRING = conf("spark.rapids.sql.json.read.mixedTypesAsString.enabled") diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index a75d2e81d71..e5b34f82973 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -18,14 +18,15 @@ package org.apache.spark.sql.catalyst.json.rapids import java.io.IOException import java.nio.charset.StandardCharsets +import java.util.Locale import scala.collection.JavaConverters._ import ai.rapids.cudf -import ai.rapids.cudf.{CaptureGroups, ColumnVector, DType, NvtxColor, RegexProgram, Scalar, Schema, Table} +import ai.rapids.cudf.{NvtxColor, Schema, Table} import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm.withResource -import com.nvidia.spark.rapids.shims.{ColumnDefaultValuesShims, GpuJsonToStructsShim, LegacyBehaviorPolicyShim, ShimFilePartitionReaderFactory} +import com.nvidia.spark.rapids.shims.{ColumnDefaultValuesShims, ShimFilePartitionReaderFactory} import org.apache.hadoop.conf.Configuration import org.apache.spark.broadcast.Broadcast @@ -40,135 +41,121 @@ import org.apache.spark.sql.execution.datasources.{PartitionedFile, Partitioning import org.apache.spark.sql.execution.datasources.v2.{FileScan, TextBasedFileScan} import org.apache.spark.sql.execution.datasources.v2.json.JsonScan import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.GpuJsonReadCommon import org.apache.spark.sql.rapids.execution.TrampolineUtil -import org.apache.spark.sql.types.{DataType, DateType, DecimalType, DoubleType, FloatType, StringType, StructType, TimestampType} +import org.apache.spark.sql.rapids.shims.GpuJsonToStructsShim +import org.apache.spark.sql.types.{DataType, DateType, DecimalType, DoubleType, FloatType, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration object GpuJsonScan { + sealed trait JsonReaderType + case object JsonScanReaderType extends JsonReaderType { + override def toString: String = "JsonScan" + } + case object JsonToStructsReaderType extends JsonReaderType { + override def toString: String = "JsonToStructs" + } + case object JsonFileFormatReaderType extends JsonReaderType { + override def toString: String = "JsonFileFormat" + } + def tagSupport(scanMeta: ScanMeta[JsonScan]) : Unit = { val scan = scanMeta.wrapped tagSupport( - scan.sparkSession, - "JsonScan", + scan.sparkSession.sessionState.conf, + JsonScanReaderType, scan.dataSchema, scan.readDataSchema, scan.options.asScala.toMap, scanMeta) } - def tagSupportOptions(opName: String, + def tagSupportOptions(op: JsonReaderType, options: JSONOptionsInRead, meta: RapidsMeta[_, _, _]): Unit = { if (options.multiLine) { - meta.willNotWorkOnGpu(s"$opName does not support multiLine") + meta.willNotWorkOnGpu(s"$op does not support multiLine") } // {"name": /* hello */ "Reynold Xin"} is not supported by CUDF if (options.allowComments) { - meta.willNotWorkOnGpu(s"$opName does not support allowComments") + meta.willNotWorkOnGpu(s"$op does not support allowComments") } // {name: 'Reynold Xin'} is not supported by CUDF if (options.allowUnquotedFieldNames) { - meta.willNotWorkOnGpu(s"$opName does not support allowUnquotedFieldNames") + meta.willNotWorkOnGpu(s"$op does not support allowUnquotedFieldNames") } // {'name': 'Reynold Xin'} turning single quotes off is not supported by CUDF if (!options.allowSingleQuotes) { - meta.willNotWorkOnGpu(s"$opName does not support disabling allowSingleQuotes") + meta.willNotWorkOnGpu(s"$op does not support disabling allowSingleQuotes") } // {"name": "Cazen Lee", "price": "\$10"} is not supported by CUDF if (options.allowBackslashEscapingAnyCharacter) { - meta.willNotWorkOnGpu(s"$opName does not support allowBackslashEscapingAnyCharacter") + meta.willNotWorkOnGpu(s"$op does not support allowBackslashEscapingAnyCharacter") } // {"a":null, "b":1, "c":3.0}, Spark will drop column `a` if dropFieldIfAllNull is enabled. if (options.dropFieldIfAllNull) { - meta.willNotWorkOnGpu(s"$opName does not support dropFieldIfAllNull") + meta.willNotWorkOnGpu(s"$op does not support dropFieldIfAllNull") } if (options.parseMode != PermissiveMode) { - meta.willNotWorkOnGpu(s"$opName only supports Permissive JSON parsing") + meta.willNotWorkOnGpu(s"$op only supports Permissive JSON parsing") } if (options.lineSeparator.getOrElse("\n") != "\n") { - meta.willNotWorkOnGpu(opName + " only supports \"\\n\" as a line separator") + meta.willNotWorkOnGpu(op + " only supports \"\\n\" as a line separator") } options.encoding.foreach(enc => if (enc != StandardCharsets.UTF_8.name() && enc != StandardCharsets.US_ASCII.name()) { - meta.willNotWorkOnGpu(s"$opName only supports UTF8 or US-ASCII encoded data") + meta.willNotWorkOnGpu(s"$op only supports UTF8 or US-ASCII encoded data") }) } - def tagJsonToStructsSupport(options:Map[String, String], - dt: DataType, - meta: RapidsMeta[_, _, _]): Unit = { - val parsedOptions = new JSONOptionsInRead( - options, - SQLConf.get.sessionLocalTimeZone, - SQLConf.get.columnNameOfCorruptRecord) - - val hasDates = TrampolineUtil.dataTypeExistsRecursively(dt, _.isInstanceOf[DateType]) - if (hasDates) { - GpuJsonToStructsShim.tagDateFormatSupport(meta, - GpuJsonUtils.optionalDateFormatInRead(parsedOptions)) - } - - val hasTimestamps = TrampolineUtil.dataTypeExistsRecursively(dt, _.isInstanceOf[TimestampType]) - if (hasTimestamps) { - GpuJsonToStructsShim.tagTimestampFormatSupport(meta, - GpuJsonUtils.optionalTimestampFormatInRead(parsedOptions)) - - GpuJsonUtils.optionalTimestampFormatInRead(parsedOptions) match { - case None | Some("yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]") => - // this is fine - case timestampFormat => - meta.willNotWorkOnGpu(s"GpuJsonToStructs unsupported timestampFormat $timestampFormat") - } - } - - if (LegacyBehaviorPolicyShim.isLegacyTimeParserPolicy) { - meta.willNotWorkOnGpu("LEGACY timeParserPolicy is not supported in GpuJsonToStructs") - } - - tagSupportOptions("JsonToStructs", parsedOptions, meta) - } - - def tagSupport(sparkSession: SparkSession, - opName: String, - dataSchema: StructType, - readSchema: StructType, + def tagSupport(conf: SQLConf, + op: JsonReaderType, + dataSchema: DataType, + readSchema: DataType, options: Map[String, String], meta: RapidsMeta[_, _, _]): Unit = { val parsedOptions = new JSONOptionsInRead( options, - sparkSession.sessionState.conf.sessionLocalTimeZone, - sparkSession.sessionState.conf.columnNameOfCorruptRecord) - - if (!meta.conf.isJsonEnabled) { - meta.willNotWorkOnGpu("JSON input and output has been disabled. To enable set " + - s"${RapidsConf.ENABLE_JSON} to true") - } + conf.sessionLocalTimeZone, + conf.columnNameOfCorruptRecord) + + op match { + case JsonScanReaderType | JsonFileFormatReaderType => + if (!meta.conf.isJsonEnabled) { + meta.willNotWorkOnGpu("JSON input and output has been disabled. To enable set " + + s"${RapidsConf.ENABLE_JSON} to true") + } - if (!meta.conf.isJsonReadEnabled) { - meta.willNotWorkOnGpu("JSON input has been disabled. To enable set " + - s"${RapidsConf.ENABLE_JSON_READ} to true. Please note that, currently json reader does " + - s"not support column prune, so user must specify the full schema or just let spark to " + - s"infer the schema") + if (!meta.conf.isJsonReadEnabled) { + meta.willNotWorkOnGpu("JSON input has been disabled. To enable set " + + s"${RapidsConf.ENABLE_JSON_READ} to true.") + } + case _ => // Ignored } - tagSupportOptions(opName, parsedOptions, meta) - - val types = readSchema.map(_.dataType) - + tagSupportOptions(op, parsedOptions, meta) val hasDates = TrampolineUtil.dataTypeExistsRecursively(readSchema, _.isInstanceOf[DateType]) - if (hasDates) { + val hasTimestamps = TrampolineUtil.dataTypeExistsRecursively(readSchema, + _.isInstanceOf[TimestampType]) + val hasFloats = TrampolineUtil.dataTypeExistsRecursively(readSchema, + _.isInstanceOf[FloatType]) + val hasDoubles = TrampolineUtil.dataTypeExistsRecursively(readSchema, + _.isInstanceOf[DoubleType]) + val hasDecimals = TrampolineUtil.dataTypeExistsRecursively(readSchema, + _.isInstanceOf[DecimalType]) + if (hasDates) { GpuTextBasedDateUtils.tagCudfFormat(meta, GpuJsonUtils.dateFormatInRead(parsedOptions), parseString = true) @@ -189,7 +176,7 @@ object GpuJsonScan { } } - if (types.contains(TimestampType) || types.contains(DateType)) { + if (hasDates || hasTimestamps) { if (!GpuOverrides.isUTCTimezone(parsedOptions.zoneId)) { meta.willNotWorkOnGpu(s"Not supported timezone type ${parsedOptions.zoneId}.") } @@ -198,46 +185,53 @@ object GpuJsonScan { GpuJsonUtils.timestampFormatInRead(parsedOptions), parseString = true) } - if(GpuJsonUtils.enableDateTimeParsingFallback(parsedOptions) && - (types.contains(DateType) || types.contains(TimestampType))) { - meta.willNotWorkOnGpu(s"GpuJsonScan does not support enableDateTimeParsingFallback") + if (GpuJsonUtils.enableDateTimeParsingFallback(parsedOptions) && + (hasDates || hasTimestamps)) { + meta.willNotWorkOnGpu(s"$op does not support enableDateTimeParsingFallback") } - if (!meta.conf.isJsonFloatReadEnabled && types.contains(FloatType)) { + if (!meta.conf.isJsonFloatReadEnabled && hasFloats) { meta.willNotWorkOnGpu("JSON reading is not 100% compatible when reading floats. " + s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_FLOATS} to true.") } - if (!meta.conf.isJsonDoubleReadEnabled && types.contains(DoubleType)) { + if (!meta.conf.isJsonDoubleReadEnabled && hasDoubles) { meta.willNotWorkOnGpu("JSON reading is not 100% compatible when reading doubles. " + s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DOUBLES} to true.") } - if (!meta.conf.isJsonDecimalReadEnabled && types.exists(_.isInstanceOf[DecimalType])) { + if (!meta.conf.isJsonDecimalReadEnabled && hasDecimals) { meta.willNotWorkOnGpu("JSON reading is not 100% compatible when reading decimals. " + s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DECIMALS} to true.") } - dataSchema.getFieldIndex(parsedOptions.columnNameOfCorruptRecord).foreach { corruptFieldIndex => - val f = dataSchema(corruptFieldIndex) - if (f.dataType != StringType || !f.nullable) { - // fallback to cpu to throw exception - meta.willNotWorkOnGpu("GpuJsonScan does not support Corrupt Record which must " + - "be string type and nullable") - } + // Technically this is a problem for dates/timestamps too, but we don't support the formats + // that are impacted by the locale + if (hasDecimals && parsedOptions.locale != Locale.US) { + meta.willNotWorkOnGpu(s"decimal parsing is only supported when the local is set " + + s"to US, but we found ${parsedOptions.locale}") } - if (readSchema.length == 1 && - readSchema.head.name == parsedOptions.columnNameOfCorruptRecord) { - // fallback to cpu to throw exception - meta.willNotWorkOnGpu("GpuJsonScan does not support Corrupt Record") + dataSchema match { + case st: StructType => + if (st.fieldNames.contains(parsedOptions.columnNameOfCorruptRecord)) { + meta.willNotWorkOnGpu(s"$op does not support Corrupt Record") + } + if (ColumnDefaultValuesShims.hasExistenceDefaultValues(st)) { + meta.willNotWorkOnGpu(s"$op does not support default values in schema") + } + case _ => //Ignored } - if (ColumnDefaultValuesShims.hasExistenceDefaultValues(readSchema)) { - meta.willNotWorkOnGpu("GpuJsonScan does not support default values in schema") + readSchema match { + case st: StructType => + FileFormatChecks.tag(meta, st, JsonFormatType, ReadFileOp) + case _ => + //This is for JsonToStructs when parsing a ArrayType or a MapType. + // ArrayType is not supported yet, and MapType only deals with String to String, so we + // are good to go. In the future we might want to wrap these in a StructType so + // we can get a full set of tests. } - - FileFormatChecks.tag(meta, readSchema, JsonFormatType, ReadFileOp) } } @@ -358,35 +352,31 @@ class JsonPartitionReader( partFile, dataSchema, readDataSchema, parsedOptions.lineSeparatorInRead, maxRowsPerChunk, maxBytesPerChunk, execMetrics, HostLineBuffererFactory) { - def buildJsonOptions(parsedOptions: JSONOptions): cudf.JSONOptions = { - cudf.JSONOptions.builder() - .withRecoverWithNull(true) - .withMixedTypesAsStrings(enableMixedTypesAsString) - .withNormalizeSingleQuotes(parsedOptions.allowSingleQuotes) - .build - } + def buildJsonOptions(parsedOptions: JSONOptions): cudf.JSONOptions = + GpuJsonReadCommon.cudfJsonOptions(parsedOptions, enableMixedTypesAsString) /** * Read the host buffer to GPU table * * @param dataBuffer host buffer to be read * @param dataSize the size of host buffer - * @param cudfSchema the cudf schema of the data + * @param cudfDataSchema the cudf schema of the data * @param readDataSchema the Spark schema describing what will be read * @param hasHeader if it has header * @return table */ override def readToTable( dataBufferer: HostLineBufferer, - cudfSchema: Schema, + cudfDataSchema: Schema, readDataSchema: StructType, + cudfReadDataSchema: Schema, hasHeader: Boolean, decodeTime: GpuMetric): Table = { val jsonOpts = buildJsonOptions(parsedOptions) - val jsonTbl = JsonPartitionReader.readToTable(dataBufferer, cudfSchema, decodeTime, jsonOpts, - getFileFormatShortName, partFile) + val jsonTbl = JsonPartitionReader.readToTable(dataBufferer, cudfReadDataSchema, decodeTime, + jsonOpts, getFileFormatShortName, partFile) withResource(jsonTbl) { tbl => - val cudfColumnNames = cudfSchema.getColumnNames + val cudfColumnNames = cudfReadDataSchema.getColumnNames val columns = readDataSchema.map { field => val i = cudfColumnNames.indexOf(field.name) if (i == -1) { @@ -399,6 +389,16 @@ class JsonPartitionReader( } } + override def getCudfSchema(dataSchema: StructType): Schema = + GpuJsonReadCommon.makeSchema(dataSchema) + + override def castTableToDesiredTypes(input: Table, dataSchema: StructType): Table = { + withResource(GpuJsonReadCommon.convertTableToDesiredType(input, dataSchema, parsedOptions)) { + cols => + new Table(cols.toSeq: _*) + } + } + /** * File format short name used for logging and other things to uniquely identity * which file format is being used. @@ -444,71 +444,16 @@ class JsonPartitionReader( } } - /** - * JSON only supports unquoted lower-case "true" and "false" as valid boolean values. - */ - override def castStringToBool(input: ColumnVector): ColumnVector = { - withResource(Scalar.fromString(true.toString)) { t => - withResource(Scalar.fromNull(DType.BOOL8)) { nullBool => - withResource(ColumnVector.fromStrings(true.toString, false.toString)) { boolStrings => - withResource(input.contains(boolStrings)) { isValidBool => - withResource(input.equalTo(t)) { - isValidBool.ifElse(_, nullBool) - } - } - } - } - } - } - - override def castStringToDate(input: ColumnVector, dt: DType): ColumnVector = { - GpuJsonToStructsShim.castJsonStringToDateFromScan(input, dt, dateFormat) - } - - /** - * JSON has strict rules about valid numeric formats. See https://www.json.org/ for specification. - * - * Spark then has its own rules for supporting NaN and Infinity, which are not - * valid numbers in JSON. - */ - private def sanitizeNumbers(input: ColumnVector): ColumnVector = { - // Note that this is not 100% consistent with Spark versions prior to Spark 3.3.0 - // due to https://issues.apache.org/jira/browse/SPARK-38060 - // cuDF `isFloat` supports some inputs that are not valid JSON numbers, such as `.1`, `1.`, - // and `+1` so we use a regular expression to match valid JSON numbers instead - val jsonNumberRegexp = "^-?[0-9]+(?:\\.[0-9]+)?(?:[eE][\\-\\+]?[0-9]+)?$" - val prog = new RegexProgram(jsonNumberRegexp, CaptureGroups.NON_CAPTURE) - val isValid = if (parsedOptions.allowNonNumericNumbers) { - withResource(ColumnVector.fromStrings("NaN", "+INF", "-INF", "+Infinity", - "Infinity", "-Infinity")) { nonNumeric => - withResource(input.matchesRe(prog)) { isJsonNumber => - withResource(input.contains(nonNumeric)) { nonNumeric => - isJsonNumber.or(nonNumeric) - } - } - } - } else { - input.matchesRe(prog) - } - withResource(isValid) { _ => - withResource(Scalar.fromNull(DType.STRING)) { nullString => - isValid.ifElse(input, nullString) - } - } - } + // TODO need to rethink how we want to handle casting data from one type to another, but probably + // only after we have nested support added in. + // https://github.com/NVIDIA/spark-rapids/issues/10539 + override def castStringToBool(input: cudf.ColumnVector): cudf.ColumnVector = + throw new IllegalStateException("THIS SHOULD NOT BE CALLED") - override def castStringToFloat(input: ColumnVector, dt: DType): ColumnVector = { - withResource(sanitizeNumbers(input)) { sanitizedInput => - super.castStringToFloat(sanitizedInput, dt) - } - } + override def dateFormat: Option[String] = + throw new IllegalStateException("THIS SHOULD NOT BE CALLED") - override def castStringToDecimal(input: ColumnVector, dt: DecimalType): ColumnVector = { - withResource(sanitizeNumbers(input)) { sanitizedInput => - super.castStringToDecimal(sanitizedInput, dt) - } - } + override def timestampFormat: String = + throw new IllegalStateException("THIS SHOULD NOT BE CALLED") - override def dateFormat: Option[String] = GpuJsonUtils.optionalDateFormatInRead(parsedOptions) - override def timestampFormat: String = GpuJsonUtils.timestampFormatInRead(parsedOptions) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala index 699342b283d..2c31ccacfcb 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala @@ -24,6 +24,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.JSONOptionsInRead +import org.apache.spark.sql.catalyst.json.rapids.GpuJsonScan.JsonFileFormatReaderType import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile @@ -86,8 +87,8 @@ object GpuReadJsonFileFormat { def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { val fsse = meta.wrapped GpuJsonScan.tagSupport( - SparkShimImpl.sessionFromPlan(fsse), - "JsonFileFormat", + SparkShimImpl.sessionFromPlan(fsse).sessionState.conf, + JsonFileFormatReaderType, fsse.relation.dataSchema, fsse.output.toStructType, fsse.relation.options, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala index 46dee35559d..e4db0adccc1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala @@ -486,8 +486,9 @@ class GpuHiveDelimitedTextPartitionReader(conf: Configuration, maxBytesPerChunk, execMetrics, HostStringColBuffererFactory) { override def readToTable(dataBufferer: HostStringColBufferer, - inputFileCudfSchema: Schema, + cudfDataSchema: Schema, requestedOutputDataSchema: StructType, + cudfReadDataSchema: Schema, isFirstChunk: Boolean, decodeTime: GpuMetric): Table = { withResource(new NvtxWithMetrics(getFileFormatShortName + " decode", @@ -515,7 +516,7 @@ class GpuHiveDelimitedTextPartitionReader(conf: Configuration, withResource(Scalar.fromNull(DType.STRING)) { nullVal => // This is a bit different because we are dropping columns/etc ourselves val requiredColumnSequence = requestedOutputDataSchema.map(_.name).toList - val outputColumnNames = inputFileCudfSchema.getColumnNames + val outputColumnNames = cudfDataSchema.getColumnNames val reorderedColumns = requiredColumnSequence.safeMap { colName => val colIndex = outputColumnNames.indexOf(colName) if (splitTable.getNumberOfColumns > colIndex) { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala new file mode 100644 index 00000000000..20f3e202ced --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala @@ -0,0 +1,331 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.sql.rapids + +import java.util.Locale + +import ai.rapids.cudf.{BinaryOp, CaptureGroups, ColumnVector, ColumnView, DType, RegexProgram, Scalar, Schema, Table} +import com.nvidia.spark.rapids.{ColumnCastUtil, GpuCast, GpuColumnVector, GpuTextBasedPartitionReader} +import com.nvidia.spark.rapids.Arm.withResource +import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingArray +import com.nvidia.spark.rapids.jni.CastStrings + +import org.apache.spark.sql.catalyst.json.{GpuJsonUtils, JSONOptions} +import org.apache.spark.sql.rapids.shims.GpuJsonToStructsShim +import org.apache.spark.sql.types._ + +/** + * This is a utility method intended to provide common functionality between JsonToStructs and + * ScanJson + */ +object GpuJsonReadCommon { + private def populateSchema(dt: DataType, + name: String, builder: Schema.Builder): Unit = dt match { + case at: ArrayType => + val child = builder.addColumn(DType.LIST, name) + populateSchema(at.elementType, "element", child) + case st: StructType => + val child = builder.addColumn(DType.STRUCT, name) + for (sf <- st.fields) { + populateSchema(sf.dataType, sf.name, child) + } + case _: MapType => + throw new IllegalArgumentException("MapType is not supported yet for schema conversion") + case _ => + builder.addColumn(DType.STRING, name) + } + + /** + * Make a read schema given an input data type + * @param input the input Spark schema to convert + * @return the schema to use when reading Spark data. + */ + def makeSchema(input: StructType): Schema = { + val builder = Schema.builder + input.foreach(f => populateSchema(f.dataType, f.name, builder)) + builder.build + } + + private def isQuotedString(input: ColumnView): ColumnVector = { + withResource(Scalar.fromString("\"")) { quote => + withResource(input.startsWith(quote)) { sw => + withResource(input.endsWith(quote)) { ew => + sw.binaryOp(BinaryOp.LOGICAL_AND, ew, DType.BOOL8) + } + } + } + } + + private def stripFirstAndLastChar(input: ColumnView): ColumnVector = { + withResource(Scalar.fromInt(1)) { one => + val end = withResource(input.getCharLengths) { cc => + withResource(cc.sub(one)) { endWithNulls => + withResource(endWithNulls.isNull) { eIsNull => + eIsNull.ifElse(one, endWithNulls) + } + } + } + withResource(end) { _ => + withResource(ColumnVector.fromScalar(one, end.getRowCount.toInt)) { start => + input.substring(start, end) + } + } + } + } + + private def undoKeepQuotes(input: ColumnView): ColumnVector = { + withResource(isQuotedString(input)) { iq => + withResource(stripFirstAndLastChar(input)) { stripped => + iq.ifElse(stripped, input) + } + } + } + + private def fixupQuotedStrings(input: ColumnView): ColumnVector = { + withResource(isQuotedString(input)) { iq => + withResource(stripFirstAndLastChar(input)) { stripped => + withResource(Scalar.fromString(null)) { ns => + iq.ifElse(stripped, ns) + } + } + } + } + + private lazy val specialUnquotedFloats = + Seq("NaN", "+INF", "-INF", "+Infinity", "Infinity", "-Infinity") + private lazy val specialQuotedFloats = specialUnquotedFloats.map(s => '"'+s+'"') + private lazy val allSpecialFloats = specialUnquotedFloats ++ specialQuotedFloats + + /** + * JSON has strict rules about valid numeric formats. See https://www.json.org/ for specification. + * + * Spark then has its own rules for supporting NaN and Infinity, which are not + * valid numbers in JSON. + */ + private def sanitizeFloats(input: ColumnView, options: JSONOptions): ColumnVector = { + // Note that this is not 100% consistent with Spark versions prior to Spark 3.3.0 + // due to https://issues.apache.org/jira/browse/SPARK-38060 + // cuDF `isFloat` supports some inputs that are not valid JSON numbers, such as `.1`, `1.`, + // and `+1` so we use a regular expression to match valid JSON numbers instead + // TODO The majority of this validation needs to move to CUDF so that we can invalidate + // an entire line/row instead of a single field. + // https://github.com/NVIDIA/spark-rapids/issues/10534 + val jsonNumberRegexp = if (options.allowNumericLeadingZeros) { + "^-?[0-9]+(?:\\.[0-9]+)?(?:[eE][\\-\\+]?[0-9]+)?$" + } else { + "^-?(?:(?:[1-9][0-9]*)|0)(?:\\.[0-9]+)?(?:[eE][\\-\\+]?[0-9]+)?$" + } + val prog = new RegexProgram(jsonNumberRegexp, CaptureGroups.NON_CAPTURE) + val isValid = if (options.allowNonNumericNumbers) { + withResource(ColumnVector.fromStrings(allSpecialFloats: _*)) { nonNumeric => + withResource(input.matchesRe(prog)) { isJsonNumber => + withResource(input.contains(nonNumeric)) { nonNumeric => + isJsonNumber.or(nonNumeric) + } + } + } + } else { + input.matchesRe(prog) + } + val cleaned = withResource(isValid) { _ => + withResource(Scalar.fromNull(DType.STRING)) { nullString => + isValid.ifElse(input, nullString) + } + } + + withResource(cleaned) { _ => + if (options.allowNonNumericNumbers) { + // Need to normalize the quotes to non-quoted to parse properly + withResource(ColumnVector.fromStrings(specialQuotedFloats: _*)) { quoted => + withResource(ColumnVector.fromStrings(specialUnquotedFloats: _*)) { unquoted => + cleaned.findAndReplaceAll(quoted, unquoted) + } + } + } else { + cleaned.incRefCount() + } + } + } + + private def sanitizeInts(input: ColumnView, options: JSONOptions): ColumnVector = { + // Integer numbers cannot look like a float, so no `.` The rest of the parsing should + // handle this correctly. + // TODO The majority of this validation needs to move to CUDF so that we can invalidate + // an entire line/row instead of a single field. + // https://github.com/NVIDIA/spark-rapids/issues/10534 + val jsonNumberRegexp = if (options.allowNumericLeadingZeros) { + "^-?[0-9]+$" + } else { + "^-?(?:(?:[1-9][0-9]*)|0)$" + } + + val prog = new RegexProgram(jsonNumberRegexp, CaptureGroups.NON_CAPTURE) + withResource(input.matchesRe(prog)) { isValid => + withResource(Scalar.fromNull(DType.STRING)) { nullString => + isValid.ifElse(input, nullString) + } + } + } + + private def sanitizeQuotedDecimalInUSLocale(input: ColumnView): ColumnVector = { + // The US locale is kind of special in that it will remove the , and then parse the + // input normally + withResource(stripFirstAndLastChar(input)) { stripped => + withResource(Scalar.fromString(",")) { comma => + withResource(Scalar.fromString("")) { empty => + stripped.stringReplace(comma, empty) + } + } + } + } + + private def sanitizeUnquotedDecimal(input: ColumnView, options: JSONOptions): ColumnVector = { + // For unquoted decimal values the number has to look like it is floating point before it is + // parsed, so this follows that, but without the special cases for INF/NaN + // TODO The majority of this validation needs to move to CUDF so that we can invalidate + // an entire line/row instead of a single field. + // https://github.com/NVIDIA/spark-rapids/issues/10534 + val jsonNumberRegexp = if (options.allowNumericLeadingZeros) { + "^-?[0-9]+(?:\\.[0-9]+)?(?:[eE][\\-\\+]?[0-9]+)?$" + } else { + "^-?(?:(?:[1-9][0-9]*)|0)(?:\\.[0-9]+)?(?:[eE][\\-\\+]?[0-9]+)?$" + } + val prog = new RegexProgram(jsonNumberRegexp, CaptureGroups.NON_CAPTURE) + withResource(input.matchesRe(prog)) { isValid => + withResource(Scalar.fromNull(DType.STRING)) { nullString => + isValid.ifElse(input, nullString) + } + } + } + + private def sanitizeDecimal(input: ColumnView, options: JSONOptions): ColumnVector = { + assert(options.locale == Locale.US) + withResource(isQuotedString(input)) { isQuoted => + withResource(sanitizeUnquotedDecimal(input, options)) { unquoted => + withResource(sanitizeQuotedDecimalInUSLocale(input)) { quoted => + isQuoted.ifElse(quoted, unquoted) + } + } + } + } + + private def castStringToFloat(input: ColumnView, dt: DType, + options: JSONOptions): ColumnVector = { + withResource(sanitizeFloats(input, options)) { sanitizedInput => + CastStrings.toFloat(sanitizedInput, false, dt) + } + } + + private def castStringToDecimal(input: ColumnVector, dt: DecimalType): ColumnVector = + CastStrings.toDecimal(input, false, false, dt.precision, -dt.scale) + + private def castJsonStringToBool(input: ColumnView): ColumnVector = { + // TODO This validation needs to move to CUDF so that we can invalidate + // an entire line/row instead of a single field. + // https://github.com/NVIDIA/spark-rapids/issues/10534 + val isTrue = withResource(Scalar.fromString("true")) { trueStr => + input.equalTo(trueStr) + } + withResource(isTrue) { _ => + val isFalse = withResource(Scalar.fromString("false")) { falseStr => + input.equalTo(falseStr) + } + val falseOrNull = withResource(isFalse) { _ => + withResource(Scalar.fromBool(false)) { falseLit => + withResource(Scalar.fromNull(DType.BOOL8)) { nul => + isFalse.ifElse(falseLit, nul) + } + } + } + withResource(falseOrNull) { _ => + withResource(Scalar.fromBool(true)) { trueLit => + isTrue.ifElse(trueLit, falseOrNull) + } + } + } + } + + private def dateFormat(options: JSONOptions): Option[String] = + GpuJsonUtils.optionalDateFormatInRead(options) + + private def timestampFormat(options: JSONOptions): String = + GpuJsonUtils.timestampFormatInRead(options) + + private def convertToDesiredType(inputCv: ColumnVector, + topLevelType: DataType, + options: JSONOptions): ColumnVector = { + ColumnCastUtil.deepTransform(inputCv, Some(topLevelType)) { + case (cv, Some(BooleanType)) if cv.getType == DType.STRING => + castJsonStringToBool(cv) + case (cv, Some(DateType)) if cv.getType == DType.STRING => + withResource(fixupQuotedStrings(cv)) { fixed => + GpuJsonToStructsShim.castJsonStringToDateFromScan(fixed, DType.TIMESTAMP_DAYS, + dateFormat(options)) + } + case (cv, Some(TimestampType)) if cv.getType == DType.STRING => + withResource(fixupQuotedStrings(cv)) { fixed => + GpuTextBasedPartitionReader.castStringToTimestamp(fixed, timestampFormat(options), + DType.TIMESTAMP_MICROSECONDS) + } + case (cv, Some(StringType)) if cv.getType == DType.STRING => + undoKeepQuotes(cv) + case (cv, Some(dt: DecimalType)) if cv.getType == DType.STRING => + withResource(sanitizeDecimal(cv, options)) { tmp => + castStringToDecimal(tmp, dt) + } + case (cv, Some(dt)) if (dt == DoubleType || dt == FloatType) && cv.getType == DType.STRING => + castStringToFloat(cv, GpuColumnVector.getNonNestedRapidsType(dt), options) + case (cv, Some(dt)) + if (dt == ByteType || dt == ShortType || dt == IntegerType || dt == LongType ) && + cv.getType == DType.STRING => + withResource(sanitizeInts(cv, options)) { tmp => + CastStrings.toInteger(tmp, false, GpuColumnVector.getNonNestedRapidsType(dt)) + } + case (cv, Some(dt)) if cv.getType == DType.STRING => + GpuCast.doCast(cv, StringType, dt) + } + } + + + /** + * Convert the parsed input table to the desired output types + * @param table the table to start with + * @param desired the desired output data types + * @param options the options the user provided + * @return an array of converted column vectors in the same order as the input table. + */ + def convertTableToDesiredType(table: Table, + desired: StructType, + options: JSONOptions): Array[ColumnVector] = { + val dataTypes = desired.fields.map(_.dataType) + dataTypes.zipWithIndex.safeMap { + case (dt, i) => + convertToDesiredType(table.getColumn(i), dt, options) + } + } + + def cudfJsonOptions(options: JSONOptions, + enableMixedTypes: Boolean): ai.rapids.cudf.JSONOptions = { + ai.rapids.cudf.JSONOptions.builder() + .withRecoverWithNull(true) + .withMixedTypesAsStrings(enableMixedTypes) + .withKeepQuotes(true) + .withNormalizeSingleQuotes(options.allowSingleQuotes) + .build() + } +} diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala index b2b502e395a..cfd34770184 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala @@ -17,21 +17,20 @@ package org.apache.spark.sql.rapids import ai.rapids.cudf -import ai.rapids.cudf.{ColumnVector, ColumnView, Cuda, DataSource, DeviceMemoryBuffer, DType, HostMemoryBuffer, Scalar, Schema} -import com.nvidia.spark.rapids.{ColumnCastUtil, GpuCast, GpuColumnVector, GpuScalar, GpuUnaryExpression, HostAlloc} +import ai.rapids.cudf.{BaseDeviceMemoryBuffer, ColumnVector, ColumnView, Cuda, DataSource, DeviceMemoryBuffer, HostMemoryBuffer, Scalar} +import com.nvidia.spark.rapids.{GpuColumnVector, GpuScalar, GpuUnaryExpression, HostAlloc} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} -import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingArray import com.nvidia.spark.rapids.jni.MapUtils -import com.nvidia.spark.rapids.shims.GpuJsonToStructsShim import org.apache.commons.text.StringEscapeUtils import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, NullIntolerant, TimeZoneAwareExpression} import org.apache.spark.sql.catalyst.json.JSONOptions +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ class JsonDeviceDataSource(combined: ColumnVector) extends DataSource { - lazy val data = combined.getData - lazy val totalSize = data.getLength + lazy val data: BaseDeviceMemoryBuffer = combined.getData + lazy val totalSize: Long = data.getLength override def size(): Long = totalSize override def hostRead(offset: Long, length: Long): HostMemoryBuffer = { @@ -66,149 +65,6 @@ class JsonDeviceDataSource(combined: ColumnVector) extends DataSource { } } -object GpuJsonToStructs { - private def populateSchema(dt: DataType, - name: String, builder: Schema.Builder): Unit = dt match { - case at: ArrayType => - val child = builder.addColumn(DType.LIST, name) - populateSchema(at.elementType, "element", child) - case st: StructType => - val child = builder.addColumn(DType.STRUCT, name) - for (sf <- st.fields) { - populateSchema(sf.dataType, sf.name, child) - } - case _: MapType => - throw new IllegalArgumentException("MapType is not supported yet for schema conversion") - case _ => - builder.addColumn(DType.STRING, name) - } - - def makeSchema(input: StructType): Schema = { - val builder = Schema.builder - input.foreach(f => populateSchema(f.dataType, f.name, builder)) - builder.build - } - - private def castJsonStringToBool(input: ColumnView): ColumnVector = { - val isTrue = withResource(Scalar.fromString("true")) { trueStr => - input.equalTo(trueStr) - } - withResource(isTrue) { _ => - val isFalse = withResource(Scalar.fromString("false")) { falseStr => - input.equalTo(falseStr) - } - val falseOrNull = withResource(isFalse) { _ => - withResource(Scalar.fromBool(false)) { falseLit => - withResource(Scalar.fromNull(DType.BOOL8)) { nul => - isFalse.ifElse(falseLit, nul) - } - } - } - withResource(falseOrNull) { _ => - withResource(Scalar.fromBool(true)) { trueLit => - isTrue.ifElse(trueLit, falseOrNull) - } - } - } - } - - private def isQuotedString(input: ColumnView): ColumnVector = { - // TODO make this a custom kernel if we need it someplace else - withResource(Scalar.fromString("\"")) { quote => - withResource(input.startsWith(quote)) { sw => - withResource(input.endsWith(quote)) { ew => - sw.binaryOp(cudf.BinaryOp.LOGICAL_AND, ew, cudf.DType.BOOL8) - } - } - } - } - - private def stripFirstAndLastChar(input: ColumnView): ColumnVector = { - // TODO make this a custom kernel - withResource(Scalar.fromInt(1)) { one => - val end = withResource(input.getCharLengths) { cc => - withResource(cc.sub(one)) { endWithNulls => - withResource(endWithNulls.isNull) { eIsNull => - eIsNull.ifElse(one, endWithNulls) - } - } - } - withResource(end) { _ => - withResource(ColumnVector.fromScalar(one, end.getRowCount.toInt)) { start => - input.substring(start, end) - } - } - } - } - - private def undoKeepQuotes(input: ColumnView): ColumnVector = { - // TODO make this go away once we have decimal parsing doing the right thing for - // both cases - withResource(isQuotedString(input)) { iq => - withResource(stripFirstAndLastChar(input)) { stripped => - iq.ifElse(stripped, input) - } - } - } - - private def fixupQuotedStrings(input: ColumnView): ColumnVector = { - // TODO make this a custom kernel - withResource(isQuotedString(input)) { iq => - withResource(stripFirstAndLastChar(input)) { stripped => - withResource(Scalar.fromString(null)) { ns => - iq.ifElse(stripped, ns) - } - } - } - } - - private def convertToDesiredType(inputCv: ColumnVector, - topLevelType: DataType, - options: Map[String, String]): ColumnVector = { - ColumnCastUtil.deepTransform(inputCv, Some(topLevelType)) { - case (cv, Some(BooleanType)) if cv.getType == DType.STRING => - castJsonStringToBool(cv) - case (cv, Some(DateType)) if cv.getType == DType.STRING => - withResource(fixupQuotedStrings(cv)) { fixed => - GpuJsonToStructsShim.castJsonStringToDate(fixed, options) - } - case (cv, Some(TimestampType)) if cv.getType == DType.STRING => - withResource(fixupQuotedStrings(cv)) { fixed => - GpuJsonToStructsShim.castJsonStringToTimestamp(fixed, options) - } - case (cv, Some(StringType)) if cv.getType == DType.STRING => - undoKeepQuotes(cv) - case (cv, Some(dt: DecimalType)) if cv.getType == DType.STRING => - // This is not actually correct, but there are other follow on issues to fix this - withResource(undoKeepQuotes(cv)) { undone => - GpuCast.doCast(undone, StringType, dt) - } - case (cv, Some(FloatType)) if cv.getType == DType.STRING => - // This is not actually correct, but there are other follow on issues to fix this - withResource(undoKeepQuotes(cv)) { undone => - GpuCast.doCast(cv, StringType, FloatType) - } - case (cv, Some(DoubleType)) if cv.getType == DType.STRING => - // This is not actually correct, but there are other follow on issues to fix this - withResource(undoKeepQuotes(cv)) { undone => - GpuCast.doCast(cv, StringType, DoubleType) - } - case(cv, Some(dt)) if cv.getType == DType.STRING => - GpuCast.doCast(cv, StringType, dt) - } - } - - def convertTableToDesiredType(table: cudf.Table, - desired: StructType, - options: Map[String, String]): Array[ColumnVector] = { - val dataTypes = desired.fields.map(_.dataType) - dataTypes.zipWithIndex.safeMap { - case (dt, i) => - convertToDesiredType(table.getColumn(i), dt, options) - } - } -} - case class GpuJsonToStructs( schema: DataType, options: Map[String, String], @@ -217,9 +73,9 @@ case class GpuJsonToStructs( timeZoneId: Option[String] = None) extends GpuUnaryExpression with TimeZoneAwareExpression with ExpectsInputTypes with NullIntolerant { - import GpuJsonToStructs._ + import GpuJsonReadCommon._ - lazy val emptyRowStr = constructEmptyRow(schema) + private lazy val emptyRowStr = constructEmptyRow(schema) private def constructEmptyRow(schema: DataType): String = { schema match { @@ -293,18 +149,13 @@ case class GpuJsonToStructs( } } - private lazy val jsonOptions = { - val parsedOptions = new JSONOptions( - options, - timeZoneId.get, - "") - cudf.JSONOptions.builder() - .withRecoverWithNull(true) - .withMixedTypesAsStrings(enableMixedTypesAsString) - .withKeepQuotes(true) - .withNormalizeSingleQuotes(parsedOptions.allowSingleQuotes) - .build() - } + private lazy val parsedOptions = new JSONOptions( + options, + timeZoneId.get, + SQLConf.get.columnNameOfCorruptRecord) + + private lazy val jsonOptions = + GpuJsonReadCommon.cudfJsonOptions(parsedOptions, enableMixedTypesAsString) override protected def doColumnar(input: GpuColumnVector): cudf.ColumnVector = { schema match { @@ -340,7 +191,7 @@ case class GpuJsonToStructs( } // Step 7: turn the data into a Struct - withResource(convertTableToDesiredType(table, struct, options)) { columns => + withResource(convertTableToDesiredType(table, struct, parsedOptions)) { columns => withResource(cudf.ColumnVector.makeStruct(columns: _*)) { structData => // Step 8: put nulls back in for nulls and empty strings withResource(GpuScalar.from(null, struct)) { nullVal => diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala similarity index 90% rename from sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala rename to sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala index 2a32b691321..6edf6d2049c 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala @@ -18,13 +18,14 @@ {"spark": "312"} {"spark": "313"} spark-rapids-shim-json-lines ***/ -package com.nvidia.spark.rapids.shims +package org.apache.spark.sql.rapids.shims import ai.rapids.cudf.{ColumnVector, ColumnView, DType, Scalar} import com.nvidia.spark.rapids.{GpuCast, GpuOverrides, RapidsMeta} import com.nvidia.spark.rapids.Arm.withResource import org.apache.spark.sql.catalyst.json.GpuJsonUtils +import org.apache.spark.sql.catalyst.json.JSONOptions import org.apache.spark.sql.rapids.ExceptionTimeParserPolicy object GpuJsonToStructsShim { @@ -36,7 +37,7 @@ object GpuJsonToStructsShim { } } - def castJsonStringToDate(input: ColumnView, options: Map[String, String]): ColumnVector = { + def castJsonStringToDate(input: ColumnView, options: JSONOptions): ColumnVector = { GpuJsonUtils.optionalDateFormatInRead(options) match { case None | Some("yyyy-MM-dd") => withResource(Scalar.fromString(" ")) { space => @@ -68,11 +69,9 @@ object GpuJsonToStructsShim { } } - def tagTimestampFormatSupport(meta: RapidsMeta[_, _, _], - timestampFormat: Option[String]): Unit = {} def castJsonStringToTimestamp(input: ColumnView, - options: Map[String, String]): ColumnVector = { + options: JSONOptions): ColumnVector = { withResource(Scalar.fromString(" ")) { space => withResource(input.strip(space)) { trimmed => // from_json doesn't respect ansi mode @@ -80,4 +79,4 @@ object GpuJsonToStructsShim { } } } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala similarity index 86% rename from sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala rename to sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala index 64e91612ba3..1724ad2fc89 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala @@ -30,12 +30,13 @@ {"spark": "333"} {"spark": "334"} spark-rapids-shim-json-lines ***/ -package com.nvidia.spark.rapids.shims +package org.apache.spark.sql.rapids.shims import ai.rapids.cudf.{ColumnVector, ColumnView, DType, Scalar} import com.nvidia.spark.rapids.{DateUtils, GpuCast, GpuOverrides, RapidsMeta} import com.nvidia.spark.rapids.Arm.withResource +import org.apache.spark.sql.catalyst.json.JSONOptions import org.apache.spark.sql.rapids.ExceptionTimeParserPolicy object GpuJsonToStructsShim { @@ -44,7 +45,7 @@ object GpuJsonToStructsShim { // performs a regular cast from string to date } - def castJsonStringToDate(input: ColumnView, options: Map[String, String]): ColumnVector = { + def castJsonStringToDate(input: ColumnView, options: JSONOptions): ColumnVector = { // dateFormat is ignored in from_json in Spark 3.2.x and 3.3.x withResource(Scalar.fromString(" ")) { space => withResource(input.strip(space)) { trimmed => @@ -78,14 +79,9 @@ object GpuJsonToStructsShim { } } - def tagTimestampFormatSupport(meta: RapidsMeta[_, _, _], - timestampFormat: Option[String]): Unit = { - // timestampFormat is ignored by JsonToStructs in Spark 3.2.x and 3.3.x because it just - // performs a regular cast from string to timestamp - } def castJsonStringToTimestamp(input: ColumnView, - options: Map[String, String]): ColumnVector = { + options: JSONOptions): ColumnVector = { // legacy behavior withResource(Scalar.fromString(" ")) { space => withResource(input.strip(space)) { trimmed => diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala similarity index 86% rename from sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala rename to sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala index 299594e7d3a..5e7b28b3a7b 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala @@ -21,21 +21,21 @@ {"spark": "350"} {"spark": "351"} spark-rapids-shim-json-lines ***/ -package com.nvidia.spark.rapids.shims +package org.apache.spark.sql.rapids.shims import ai.rapids.cudf.{ColumnVector, ColumnView, DType, Scalar} import com.nvidia.spark.rapids.{DateUtils, GpuCast, GpuOverrides, RapidsMeta} import com.nvidia.spark.rapids.Arm.withResource import org.apache.spark.sql.catalyst.json.GpuJsonUtils +import org.apache.spark.sql.catalyst.json.JSONOptions import org.apache.spark.sql.rapids.ExceptionTimeParserPolicy object GpuJsonToStructsShim { - def tagDateFormatSupport(meta: RapidsMeta[_, _, _], dateFormat: Option[String]): Unit = { } - def castJsonStringToDate(input: ColumnView, options: Map[String, String]): ColumnVector = { + def castJsonStringToDate(input: ColumnView, options: JSONOptions): ColumnVector = { GpuJsonUtils.optionalDateFormatInRead(options) match { case None => // legacy behavior @@ -78,15 +78,9 @@ object GpuJsonToStructsShim { GpuCast.convertDateOrNull(input, "^" + regexRoot + "$", cudfFormat, failOnInvalid) } - def tagTimestampFormatSupport(meta: RapidsMeta[_, _, _], - timestampFormat: Option[String]): Unit = { - // we only support the case where no format is specified - timestampFormat.foreach(f => meta.willNotWorkOnGpu(s"Unsupported timestampFormat: $f")) - } - def castJsonStringToTimestamp(input: ColumnView, - options: Map[String, String]): ColumnVector = { - options.get("timestampFormat") match { + options: JSONOptions): ColumnVector = { + options.timestampFormatInRead match { case None => // legacy behavior withResource(Scalar.fromString(" ")) { space =>