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 38a1ec18524..79f8ee44792 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 @@ -119,6 +119,10 @@ object GpuJsonScan { 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 diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala index 1580659c2b4..e5876e13ff1 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala @@ -37,7 +37,6 @@ object GpuJsonToStructsShim { def tagDateFormatSupport(meta: RapidsMeta[_, _, _], dateFormat: Option[String]): Unit = { dateFormat match { case None | Some("yyyy-MM-dd") => - // this is fine case dateFormat => meta.willNotWorkOnGpu(s"GpuJsonToStructs unsupported dateFormat $dateFormat") } @@ -74,6 +73,9 @@ object GpuJsonToStructsShim { } } + def tagTimestampFormatSupport(meta: RapidsMeta[_, _, _], + timestampFormat: Option[String]): Unit = {} + def castJsonStringToTimestamp(input: ColumnVector, options: Map[String, String]): ColumnVector = { withResource(Scalar.fromString(" ")) { space => diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala index 110cfbd6a76..3e723f93548 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala @@ -29,7 +29,7 @@ import ai.rapids.cudf.{ColumnVector, DType, Scalar} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.{DateUtils, GpuCast, GpuOverrides, RapidsMeta} -import org.apache.spark.sql.catalyst.json.GpuJsonUtils +//import org.apache.spark.sql.catalyst.json.GpuJsonUtils import org.apache.spark.sql.rapids.ExceptionTimeParserPolicy object GpuJsonToStructsShim { @@ -38,17 +38,11 @@ object GpuJsonToStructsShim { } def castJsonStringToDate(input: ColumnVector, options: Map[String, String]): ColumnVector = { - GpuJsonUtils.optionalDateFormatInRead(options) match { - case None => - // legacy behavior - withResource(Scalar.fromString(" ")) { space => - withResource(input.strip(space)) { trimmed => - GpuCast.castStringToDate(trimmed) - } - } - case Some(f) => - // from_json does not respect EXCEPTION policy - jsonStringToDate(input, f, failOnInvalid = false) + // dateFormat is ignored in Spark 3.3 + withResource(Scalar.fromString(" ")) { space => + withResource(input.strip(space)) { trimmed => + GpuCast.castStringToDate(trimmed) + } } } @@ -56,7 +50,7 @@ object GpuJsonToStructsShim { } def castJsonStringToDateFromScan(input: ColumnVector, dt: DType, dateFormat: Option[String], - failOnInvalid: Boolean): ColumnVector = { + failOnInvalid: Boolean): ColumnVector = { dateFormat match { case None => // legacy behavior @@ -64,8 +58,10 @@ object GpuJsonToStructsShim { GpuCast.castStringToDateAnsi(trimmed, ansiMode = false) } case Some(f) => - jsonStringToDate(input, f, failOnInvalid && - GpuOverrides.getTimeParserPolicy == ExceptionTimeParserPolicy) + withResource(input.strip()) { trimmed => + jsonStringToDate(trimmed, f, failOnInvalid && + GpuOverrides.getTimeParserPolicy == ExceptionTimeParserPolicy) + } } } @@ -73,31 +69,23 @@ object GpuJsonToStructsShim { failOnInvalid: Boolean): ColumnVector = { val regexRoot = dateFormatPattern .replace("yyyy", raw"\d{4}") - .replace("MM", raw"\d{2}") - .replace("dd", raw"\d{2}") + .replace("MM", raw"\d{1,2}") + .replace("dd", raw"\d{1,2}") val cudfFormat = DateUtils.toStrf(dateFormatPattern, parseString = true) - withResource(input.strip()) { input => - GpuCast.convertDateOrNull(input, "^" + regexRoot + "$", cudfFormat, failOnInvalid) - } + GpuCast.convertDateOrNull(input, "^" + regexRoot + "$", cudfFormat, failOnInvalid) } + def tagTimestampFormatSupport(meta: RapidsMeta[_, _, _], + timestampFormat: Option[String]): Unit = {} + def castJsonStringToTimestamp(input: ColumnVector, options: Map[String, String]): ColumnVector = { - options.get("timestampFormat") match { - case None => - // legacy behavior - withResource(Scalar.fromString(" ")) { space => - withResource(input.strip(space)) { trimmed => - // from_json doesn't respect ansi mode - GpuCast.castStringToTimestamp(trimmed, ansiMode = false) - } - } - case Some("yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]") => - GpuCast.convertTimestampOrNull(input, - "^[0-9]{4}-[0-9]{2}-[0-9]{2}T[0-9]{2}:[0-9]{2}:[0-9]{2}(\\.[0-9]{1,6})?Z?$", "%Y-%m-%d") - case other => - // should be unreachable due to GpuOverrides checks - throw new IllegalStateException(s"Unsupported timestampFormat $other") + // legacy behavior + withResource(Scalar.fromString(" ")) { space => + withResource(input.strip(space)) { trimmed => + // from_json doesn't respect ansi mode + GpuCast.castStringToTimestamp(trimmed, ansiMode = false) + } } } } \ No newline at end of file diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala index af5ba2674a3..8bd3681b01d 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuJsonToStructsShim.scala @@ -66,6 +66,7 @@ object GpuJsonToStructsShim { } } + private def jsonStringToDate(input: ColumnVector, dateFormatPattern: String, failOnInvalid: Boolean): ColumnVector = { val regexRoot = dateFormatPattern @@ -76,6 +77,16 @@ object GpuJsonToStructsShim { GpuCast.convertDateOrNull(input, "^" + regexRoot + "$", cudfFormat, failOnInvalid) } + def tagTimestampFormatSupport(meta: RapidsMeta[_, _, _], + timestampFormat: Option[String]): Unit = { + timestampFormat match { + case None | Some("yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]") => + // fine + case other => + meta.willNotWorkOnGpu(s"Unsupported timestampFormat ${other}") + } + } + def castJsonStringToTimestamp(input: ColumnVector, options: Map[String, String]): ColumnVector = { options.get("timestampFormat") match {