diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 44d2a56d413..3088bec714b 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -236,8 +236,7 @@ def test_basic_json_read(std_input_path, filename, schema, read_func, allow_non_ @pytest.mark.parametrize('read_func', [read_json_df, read_json_sql]) @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) @pytest.mark.parametrize('time_parser_policy', [ - pytest.param('LEGACY', marks=[pytest.mark.allow_non_gpu('FileSourceScanExec'), \ - pytest.mark.xfail(is_spark_330_or_later(), reason="https://github.com/NVIDIA/spark-rapids/issues/4912")]), + pytest.param('LEGACY', marks=pytest.mark.allow_non_gpu('FileSourceScanExec')), 'CORRECTED', 'EXCEPTION' ]) @@ -263,8 +262,7 @@ def test_json_read_valid_dates(std_input_path, filename, schema, read_func, ansi @pytest.mark.parametrize('read_func', [read_json_df, read_json_sql]) @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) @pytest.mark.parametrize('time_parser_policy', [ - pytest.param('LEGACY', marks=[pytest.mark.allow_non_gpu('FileSourceScanExec'), \ - pytest.mark.xfail(is_spark_330_or_later(), reason="https://github.com/NVIDIA/spark-rapids/issues/4912")]), + pytest.param('LEGACY', marks=pytest.mark.allow_non_gpu('FileSourceScanExec')), 'CORRECTED', 'EXCEPTION' ]) diff --git a/sql-plugin/src/main/301until330-all/scala/org/apache/spark/sql/catalyst/json/rapids/shims/v2/Spark30Xuntil33XFileOptionsShims.scala b/sql-plugin/src/main/301until330-all/scala/org/apache/spark/sql/catalyst/json/rapids/shims/v2/Spark30Xuntil33XFileOptionsShims.scala index b68404a5887..8789ceaf287 100644 --- a/sql-plugin/src/main/301until330-all/scala/org/apache/spark/sql/catalyst/json/rapids/shims/v2/Spark30Xuntil33XFileOptionsShims.scala +++ b/sql-plugin/src/main/301until330-all/scala/org/apache/spark/sql/catalyst/json/rapids/shims/v2/Spark30Xuntil33XFileOptionsShims.scala @@ -23,14 +23,6 @@ import org.apache.spark.sql.catalyst.json.JSONOptions trait Spark30Xuntil33XFileOptionsShims extends SparkShims { - def dateFormatInRead(fileOptions: Serializable): Option[String] = { - fileOptions match { - case csvOpts: CSVOptions => Option(csvOpts.dateFormat) - case jsonOpts: JSONOptions => Option(jsonOpts.dateFormat) - case _ => throw new RuntimeException("Wrong file options.") - } - } - def timestampFormatInRead(fileOptions: Serializable): Option[String] = { fileOptions match { case csvOpts: CSVOptions => Option(csvOpts.timestampFormat) diff --git a/sql-plugin/src/main/330+/scala/org/apache/spark/sql/catalyst/json/rapids/shims/v2/Spark33XFileOptionsShims.scala b/sql-plugin/src/main/330+/scala/org/apache/spark/sql/catalyst/json/rapids/shims/v2/Spark33XFileOptionsShims.scala index b762f5975a8..5c1636a994b 100644 --- a/sql-plugin/src/main/330+/scala/org/apache/spark/sql/catalyst/json/rapids/shims/v2/Spark33XFileOptionsShims.scala +++ b/sql-plugin/src/main/330+/scala/org/apache/spark/sql/catalyst/json/rapids/shims/v2/Spark33XFileOptionsShims.scala @@ -23,14 +23,6 @@ import org.apache.spark.sql.catalyst.json.JSONOptions trait Spark33XFileOptionsShims extends Spark321PlusShims { - def dateFormatInRead(fileOptions: Serializable): Option[String] = { - fileOptions match { - case csvOpts: CSVOptions => csvOpts.dateFormatInRead - case jsonOpts: JSONOptions => jsonOpts.dateFormatInRead - case _ => throw new RuntimeException("Wrong file options.") - } - } - def timestampFormatInRead(fileOptions: Serializable): Option[String] = { fileOptions match { case csvOpts: CSVOptions => csvOpts.dateFormatInRead diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala index 9471c4b8dee..5845c975838 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala @@ -218,9 +218,8 @@ object GpuCSVScan { // so we fall back to CPU meta.willNotWorkOnGpu(s"GpuCSVScan does not support timeParserPolicy=LEGACY") } - ShimLoader.getSparkShims.dateFormatInRead(parsedOptions).foreach { dateFormat => - DateUtils.tagAndGetCudfFormat(meta, dateFormat, parseString = true) - } + DateUtils.tagAndGetCudfFormat(meta, + GpuCsvUtils.dateFormatInRead(parsedOptions), parseString = true) } if (readSchema.map(_.dataType).contains(TimestampType)) { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index 8b29932e014..eba6798b38b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -303,9 +303,6 @@ trait SparkShims { */ def getLegacyStatisticalAggregate(): Boolean - - def dateFormatInRead(fileOptions: Serializable): Option[String] - def timestampFormatInRead(fileOptions: Serializable): Option[String] def neverReplaceShowCurrentNamespaceCommand: ExecRule[_ <: SparkPlan] 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 64810bf9551..a6ebdb40ee1 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 @@ -147,9 +147,8 @@ object GpuJsonScan { }) if (readSchema.map(_.dataType).contains(DateType)) { - ShimLoader.getSparkShims.dateFormatInRead(parsedOptions).foreach { dateFormat => - DateUtils.tagAndGetCudfFormat(meta, dateFormat, parseString = true) - } + DateUtils.tagAndGetCudfFormat(meta, + GpuJsonUtils.dateFormatInRead(parsedOptions), parseString = true) } if (readSchema.map(_.dataType).contains(TimestampType)) {