forked from NVIDIA/spark-rapids
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Improve dateFormat support in GpuJsonScan and make tests consistent w…
…ith GpuStructsToJson [databricks] (NVIDIA#9975) * upmerge * Revert change to csv_test Signed-off-by: Andy Grove <[email protected]> * scalastyle * introduce shim * remove unreachable code * fix some failures with 311 * save progress * fix more failures with 340 * fix test failures with 341 * tests pass with 341 * Add 330 shim and fix failures in test_basic_json_read * save progress on 330 shim * tests pass with 330 * 320 shim * test all date formats with from_json * remove redundant and confusing use of failOnInvalid parameter * Revert unrelated change * Remove comment * Remove blank line * Remove blank line * Revert accidental change to test_basic_json_read * Fix compilation error caused by refactor * Scala style * Scala style * update compatibility guide * move json-specific date parsing into GpuJsonScan to fix regression in CSV tests * fix regression introduced during refactor * fall back to CPU if timestampFormat specified in 320 shim * fall back to CPU if timestampFormat specified in 340 shim * fix ci failure with 341db * add 334 shim * add clarifying comment * update copyright years to 2024 * use None instead of empty string in tests * fix copyright years * remove xfail from tests * fix regression * Update integration_tests/src/main/python/json_test.py Co-authored-by: Jason Lowe <[email protected]> * Update integration_tests/src/main/python/json_test.py Co-authored-by: Jason Lowe <[email protected]> * fix regression * Revert "fix regression" This reverts commit 4b183a4. * update more tests to use None instead of empty string * allow fallback for non-utc in test_json_read_generated_dates * update more tests to use None instead of empty string --------- Signed-off-by: Andy Grove <[email protected]> Co-authored-by: Jason Lowe <[email protected]>
- Loading branch information
Showing
9 changed files
with
391 additions
and
156 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Large diffs are not rendered by default.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
99 changes: 99 additions & 0 deletions
99
sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,99 @@ | ||
/* | ||
* 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. | ||
*/ | ||
/*** spark-rapids-shim-json-lines | ||
{"spark": "320"} | ||
{"spark": "321"} | ||
{"spark": "321cdh"} | ||
{"spark": "321db"} | ||
{"spark": "322"} | ||
{"spark": "323"} | ||
{"spark": "324"} | ||
{"spark": "330"} | ||
{"spark": "330cdh"} | ||
{"spark": "330db"} | ||
{"spark": "331"} | ||
{"spark": "332"} | ||
{"spark": "332cdh"} | ||
{"spark": "332db"} | ||
{"spark": "333"} | ||
{"spark": "334"} | ||
spark-rapids-shim-json-lines ***/ | ||
package com.nvidia.spark.rapids.shims | ||
|
||
import ai.rapids.cudf.{ColumnVector, DType, Scalar} | ||
import com.nvidia.spark.rapids.{DateUtils, GpuCast, GpuOverrides, RapidsMeta} | ||
import com.nvidia.spark.rapids.Arm.withResource | ||
|
||
import org.apache.spark.sql.rapids.ExceptionTimeParserPolicy | ||
|
||
object GpuJsonToStructsShim { | ||
|
||
def tagDateFormatSupport(meta: RapidsMeta[_, _, _], dateFormat: Option[String]): Unit = { | ||
// dateFormat is ignored by JsonToStructs in Spark 3.2.x and 3.3.x because it just | ||
// performs a regular cast from string to date | ||
} | ||
|
||
def castJsonStringToDate(input: ColumnVector, options: Map[String, String]): ColumnVector = { | ||
// dateFormat is ignored in from_json in Spark 3.2 | ||
withResource(Scalar.fromString(" ")) { space => | ||
withResource(input.strip(space)) { trimmed => | ||
GpuCast.castStringToDate(trimmed) | ||
} | ||
} | ||
} | ||
|
||
def tagDateFormatSupportFromScan(meta: RapidsMeta[_, _, _], dateFormat: Option[String]): Unit = { | ||
} | ||
|
||
def castJsonStringToDateFromScan(input: ColumnVector, dt: DType, | ||
dateFormat: Option[String]): ColumnVector = { | ||
dateFormat match { | ||
case None => | ||
// legacy behavior | ||
withResource(input.strip()) { trimmed => | ||
GpuCast.castStringToDateAnsi(trimmed, ansiMode = | ||
GpuOverrides.getTimeParserPolicy == ExceptionTimeParserPolicy) | ||
} | ||
case Some(fmt) => | ||
withResource(input.strip()) { trimmed => | ||
val regexRoot = fmt | ||
.replace("yyyy", raw"\d{4}") | ||
.replace("MM", raw"\d{1,2}") | ||
.replace("dd", raw"\d{1,2}") | ||
val cudfFormat = DateUtils.toStrf(fmt, parseString = true) | ||
GpuCast.convertDateOrNull(trimmed, "^" + regexRoot + "$", cudfFormat, | ||
failOnInvalid = GpuOverrides.getTimeParserPolicy == ExceptionTimeParserPolicy) | ||
} | ||
} | ||
} | ||
|
||
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: ColumnVector, | ||
options: Map[String, String]): ColumnVector = { | ||
// legacy behavior | ||
withResource(Scalar.fromString(" ")) { space => | ||
withResource(input.strip(space)) { trimmed => | ||
// from_json doesn't respect ansi mode | ||
GpuCast.castStringToTimestamp(trimmed, ansiMode = false) | ||
} | ||
} | ||
} | ||
} |
Oops, something went wrong.