This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 1a7a2f7a889f [SPARK-46488][SQL] Skipping trimAll call during timestamp parsing 1a7a2f7a889f is described below commit 1a7a2f7a889fe0270318b304cd50c148729dd90b Author: Stefan Kandic <stefan.kan...@databricks.com> AuthorDate: Mon Dec 25 19:41:10 2023 +0300 [SPARK-46488][SQL] Skipping trimAll call during timestamp parsing ### What changes were proposed in this pull request? This PR is a follow up to [46173](https://github.com/apache/spark/pull/44110) which added skipping the trimAll calls during date parsing. Now I'm doing the same just for timestamp parsing. ### Why are the changes needed? These changes should drastically improve edge case where input string in cast to date has many whitespace as prefix/sufix. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? I added the tests to check for cases with prefixes and suffixes of whitespaces and control chars. Also there are benchmark tests in the previous [PR](https://github.com/apache/spark/pull/44110) ### Was this patch authored or co-authored using generative AI tooling? No Closes #44463 from stefankandic/str2timeStamp-skipTrim. Authored-by: Stefan Kandic <stefan.kan...@databricks.com> Signed-off-by: Max Gekk <max.g...@gmail.com> --- .../sql/catalyst/util/SparkDateTimeUtils.scala | 66 ++++++++++++++++------ .../sql/catalyst/util/DateTimeUtilsSuite.scala | 43 +++++++++----- 2 files changed, 79 insertions(+), 30 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala index 35118b449e2f..ed4d68f553f1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala @@ -315,18 +315,11 @@ trait SparkDateTimeUtils { var currentSegmentValue = 0 var currentSegmentDigits = 0 val bytes = s.getBytes - var j = 0 - var strEndTrimmed = bytes.length + var j = getTrimmedStart(bytes) + val strEndTrimmed = getTrimmedEnd(j, bytes) - while (j < bytes.length && UTF8String.isWhitespaceOrISOControl(bytes(j))) { - j += 1; - } - if (j == bytes.length) { - return None; - } - - while (strEndTrimmed > j && UTF8String.isWhitespaceOrISOControl(bytes(strEndTrimmed - 1))) { - strEndTrimmed -= 1; + if (j == strEndTrimmed) { + return None } if (bytes(j) == '-' || bytes(j) == '+') { @@ -418,7 +411,7 @@ trait SparkDateTimeUtils { (segment == 7 && digits <= 2) || (segment != 0 && segment != 6 && segment != 7 && digits > 0 && digits <= 2) } - if (s == null || s.trimAll().numBytes() == 0) { + if (s == null) { return (Array.empty, None, false) } var tz: Option[String] = None @@ -426,8 +419,14 @@ trait SparkDateTimeUtils { var i = 0 var currentSegmentValue = 0 var currentSegmentDigits = 0 - val bytes = s.trimAll().getBytes - var j = 0 + val bytes = s.getBytes + var j = getTrimmedStart(bytes) + val strEndTrimmed = getTrimmedEnd(j, bytes) + + if (j == strEndTrimmed) { + return (Array.empty, None, false) + } + var digitsMilli = 0 var justTime = false var yearSign: Option[Int] = None @@ -435,7 +434,7 @@ trait SparkDateTimeUtils { yearSign = if (bytes(j) == '-') Some(-1) else Some(1) j += 1 } - while (j < bytes.length) { + while (j < strEndTrimmed) { val b = bytes(j) val parsedValue = b - '0'.toByte if (parsedValue < 0 || parsedValue > 9) { @@ -504,8 +503,8 @@ trait SparkDateTimeUtils { currentSegmentValue = 0 currentSegmentDigits = 0 i += 1 - tz = Some(new String(bytes, j, bytes.length - j)) - j = bytes.length - 1 + tz = Some(new String(bytes, j, strEndTrimmed - j)) + j = strEndTrimmed - 1 } if (i == 6 && b != '.') { i += 1 @@ -619,6 +618,39 @@ trait SparkDateTimeUtils { case NonFatal(_) => None } } + + /** + * Returns the index of the first non-whitespace and non-ISO control character in the byte array. + * + * @param bytes The byte array to be processed. + * @return The start index after trimming. + */ + @inline private def getTrimmedStart(bytes: Array[Byte]) = { + var start = 0 + + while (start < bytes.length && UTF8String.isWhitespaceOrISOControl(bytes(start))) { + start += 1 + } + + start + } + + /** + * Returns the index of the last non-whitespace and non-ISO control character in the byte array. + * + * @param start The starting index for the search. + * @param bytes The byte array to be processed. + * @return The end index after trimming. + */ + @inline private def getTrimmedEnd(start: Int, bytes: Array[Byte]) = { + var end = bytes.length - 1 + + while (end > start && UTF8String.isWhitespaceOrISOControl(bytes(end))) { + end -= 1 + } + + end + 1 + } } object SparkDateTimeUtils extends SparkDateTimeUtils diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index c4e578ee2a51..c4f6e4d6ed0e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -128,23 +128,26 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } test("SPARK-32559: string to date trim Control Characters") { - Seq("2015-03-18", "2015-03-18T123321", " 2015-03-18 123321", "+2015-03-18") - .foreach { - input => Seq(input, "\u0003", "\u0003", " ", " ") - .permutations.map(p => p.mkString).foreach { - s => assert(toDate(s).get === days(2015, 3, 18)) - } + val expected = days(2015, 3, 18) + permuteWithWhitespaceAndControl( + "2015-03-18", "2015-03-18T123321", " 2015-03-18 123321", "+2015-03-18" + ).foreach { s => + assert(toDate(s).get === expected) } - Seq("INVALID_INPUT", " ", "1999-08-", "2015-03-18\u0003123321", "2015-03-18Q123321") - .foreach { - input => - Seq(input, "\u0003", "\u0003", " ", " ").permutations.map(p => p.mkString).foreach { - s => assert(toDate(s).isEmpty) - } - } + permuteWithWhitespaceAndControl( + "INVALID_INPUT", " ", "1999-08-", "2015-03-18\u0003123321", "2015-03-18Q123321" + ).foreach { s => + assert(toDate(s).isEmpty) + } } + private def permuteWithWhitespaceAndControl(values: String*): Seq[String] = + values.flatMap { input => + Seq(input, "\u0003", "\u0003", " ", " ") + .permutations.map(_.mkString) + } + test("string to date") { assert(toDate("2015-01-28").get === days(2015, 1, 28)) assert(toDate("2015").get === days(2015, 1, 1)) @@ -337,6 +340,20 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { zoneId = getZoneId("Europe/Moscow") expected = Option(date(2015, 3, 18, 12, 3, 17, 123456, zid = zoneId)) checkStringToTimestamp("2015-03-18T12:03:17.123456 Europe/Moscow", expected) + + // Check whitespace and control character permutations + expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zid)) + permuteWithWhitespaceAndControl( + "2015-03-18 12:03:17", "2015-03-18T12:03:17" + ).foreach { s => + checkStringToTimestamp(s, expected) + } + + permuteWithWhitespaceAndControl( + "INVALID_INPUT", "\t", "", "2015-03-18\u000312:03:17", "2015-03-18 12:", "2015-03-18 123" + ).foreach { s => + checkStringToTimestamp(s, None) + } } } --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org