Skip to content

Commit 3206a99

Browse files
MaxGekkcloud-fan
authored andcommitted
[SPARK-29651][SQL] Fix parsing of interval seconds fraction
### What changes were proposed in this pull request? In the PR, I propose to extract parsing of the seconds interval units to the private method `parseNanos` in `IntervalUtils` and modify the code to correctly parse the fractional part of the seconds unit of intervals in the cases: - When the fractional part has less than 9 digits - The seconds unit is negative ### Why are the changes needed? The changes are needed to fix the issues: ```sql spark-sql> select interval '10.123456 seconds'; interval 10 seconds 123 microseconds ``` The correct result must be `interval 10 seconds 123 milliseconds 456 microseconds` ```sql spark-sql> select interval '-10.123456789 seconds'; interval -9 seconds -876 milliseconds -544 microseconds ``` but the whole interval should be negated, and the result must be `interval -10 seconds -123 milliseconds -456 microseconds`, taking into account the truncation to microseconds. ### Does this PR introduce any user-facing change? Yes. After changes: ```sql spark-sql> select interval '10.123456 seconds'; interval 10 seconds 123 milliseconds 456 microseconds spark-sql> select interval '-10.123456789 seconds'; interval -10 seconds -123 milliseconds -456 microseconds ``` ### How was this patch tested? By existing and new tests in `ExpressionParserSuite`. Closes #26313 from MaxGekk/fix-interval-nanos-parsing. Authored-by: Maxim Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent cfc80d0 commit 3206a99

File tree

3 files changed

+36
-18
lines changed

3 files changed

+36
-18
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala

Lines changed: 22 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -218,27 +218,22 @@ object IntervalUtils {
218218
minutes = toLongWithRange("second", m.group(7), 0, 59)
219219
}
220220
// Hive allow nanosecond precision interval
221-
val nanoStr = if (m.group(9) == null) {
222-
null
223-
} else {
224-
(m.group(9) + "000000000").substring(0, 9)
225-
}
226-
var nanos = toLongWithRange("nanosecond", nanoStr, 0L, 999999999L)
221+
var secondsFraction = parseNanos(m.group(9), seconds < 0)
227222
to match {
228223
case "hour" =>
229224
minutes = 0
230225
seconds = 0
231-
nanos = 0
226+
secondsFraction = 0
232227
case "minute" =>
233228
seconds = 0
234-
nanos = 0
229+
secondsFraction = 0
235230
case "second" =>
236231
// No-op
237232
case _ =>
238233
throw new IllegalArgumentException(
239234
s"Cannot support (interval '$input' $from to $to) expression")
240235
}
241-
var micros = nanos / DateTimeUtils.NANOS_PER_MICROS
236+
var micros = secondsFraction
242237
micros = Math.addExact(micros, Math.multiplyExact(days, DateTimeUtils.MICROS_PER_DAY))
243238
micros = Math.addExact(micros, Math.multiplyExact(hours, MICROS_PER_HOUR))
244239
micros = Math.addExact(micros, Math.multiplyExact(minutes, MICROS_PER_MINUTE))
@@ -292,6 +287,21 @@ object IntervalUtils {
292287
new CalendarInterval(months, microseconds)
293288
}
294289

290+
// Parses a string with nanoseconds, truncates the result and returns microseconds
291+
private def parseNanos(nanosStr: String, isNegative: Boolean): Long = {
292+
if (nanosStr != null) {
293+
val maxNanosLen = 9
294+
val alignedStr = if (nanosStr.length < maxNanosLen) {
295+
(nanosStr + "000000000").substring(0, maxNanosLen)
296+
} else nanosStr
297+
val nanos = toLongWithRange("nanosecond", alignedStr, 0L, 999999999L)
298+
val micros = nanos / DateTimeUtils.NANOS_PER_MICROS
299+
if (isNegative) -micros else micros
300+
} else {
301+
0L
302+
}
303+
}
304+
295305
/**
296306
* Parse second_nano string in ss.nnnnnnnnn format to microseconds
297307
*/
@@ -303,15 +313,13 @@ object IntervalUtils {
303313
Long.MinValue / DateTimeUtils.MICROS_PER_SECOND,
304314
Long.MaxValue / DateTimeUtils.MICROS_PER_SECOND) * DateTimeUtils.MICROS_PER_SECOND
305315
}
306-
def parseNanos(nanosStr: String): Long = {
307-
toLongWithRange("nanosecond", nanosStr, 0L, 999999999L) / DateTimeUtils.NANOS_PER_MICROS
308-
}
309316

310317
secondNano.split("\\.") match {
311318
case Array(secondsStr) => parseSeconds(secondsStr)
312-
case Array("", nanosStr) => parseNanos(nanosStr)
319+
case Array("", nanosStr) => parseNanos(nanosStr, false)
313320
case Array(secondsStr, nanosStr) =>
314-
Math.addExact(parseSeconds(secondsStr), parseNanos(nanosStr))
321+
val seconds = parseSeconds(secondsStr)
322+
Math.addExact(seconds, parseNanos(nanosStr, seconds < 0))
315323
case _ =>
316324
throw new IllegalArgumentException(
317325
"Interval string does not match second-nano format of ss.nnnnnnnnn")

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier
2424
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
2525
import org.apache.spark.sql.catalyst.expressions._
2626
import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last}
27-
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils}
27+
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, IntervalUtils}
2828
import org.apache.spark.sql.internal.SQLConf
2929
import org.apache.spark.sql.types._
3030
import org.apache.spark.unsafe.types.CalendarInterval
@@ -628,7 +628,17 @@ class ExpressionParserSuite extends AnalysisTest {
628628

629629
// Hive nanosecond notation.
630630
checkIntervals("13.123456789 seconds", intervalLiteral("second", "13.123456789"))
631-
checkIntervals("-13.123456789 second", intervalLiteral("second", "-13.123456789"))
631+
checkIntervals(
632+
"-13.123456789 second",
633+
Literal(new CalendarInterval(
634+
0,
635+
-13 * DateTimeUtils.MICROS_PER_SECOND - 123 * DateTimeUtils.MICROS_PER_MILLIS - 456)))
636+
checkIntervals(
637+
"13.123456 second",
638+
Literal(new CalendarInterval(
639+
0,
640+
13 * DateTimeUtils.MICROS_PER_SECOND + 123 * DateTimeUtils.MICROS_PER_MILLIS + 456)))
641+
checkIntervals("1.001 second", Literal(IntervalUtils.fromString("1 second 1 millisecond")))
632642

633643
// Non Existing unit
634644
intercept("interval 10 nanoseconds",

sql/core/src/test/resources/sql-tests/results/literals.sql.out

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -323,9 +323,9 @@ select timestamp '2016-33-11 20:54:00.000'
323323
-- !query 34
324324
select interval 13.123456789 seconds, interval -13.123456789 second
325325
-- !query 34 schema
326-
struct<interval 13 seconds 123 milliseconds 456 microseconds:interval,interval -12 seconds -876 milliseconds -544 microseconds:interval>
326+
struct<interval 13 seconds 123 milliseconds 456 microseconds:interval,interval -13 seconds -123 milliseconds -456 microseconds:interval>
327327
-- !query 34 output
328-
interval 13 seconds 123 milliseconds 456 microseconds interval -12 seconds -876 milliseconds -544 microseconds
328+
interval 13 seconds 123 milliseconds 456 microseconds interval -13 seconds -123 milliseconds -456 microseconds
329329

330330

331331
-- !query 35

0 commit comments

Comments
 (0)