Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1278,10 +1278,17 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
}

/** Create a numeric literal expression. */
private def numericLiteral(ctx: NumberContext)(f: String => Any): Literal = withOrigin(ctx) {
val raw = ctx.getText
private def numericLiteral
(ctx: NumberContext, minValue: BigDecimal, maxValue: BigDecimal, typeName: String)
(converter: String => Any): Literal = withOrigin(ctx) {
val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1)
try {
Literal(f(raw.substring(0, raw.length - 1)))
val rawBigDecimal = BigDecimal(rawStrippedQualifier)
if (rawBigDecimal < minValue || rawBigDecimal > maxValue) {
throw new ParseException(s"Numeric literal ${rawStrippedQualifier} does not " +
s"fit in range [${minValue}, ${maxValue}] for type ${typeName}", ctx)
}
Literal(converter(rawStrippedQualifier))
} catch {
case e: NumberFormatException =>
throw new ParseException(e.getMessage, ctx)
Expand All @@ -1291,29 +1298,29 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
/**
* Create a Byte Literal expression.
*/
override def visitTinyIntLiteral(ctx: TinyIntLiteralContext): Literal = numericLiteral(ctx) {
_.toByte
override def visitTinyIntLiteral(ctx: TinyIntLiteralContext): Literal = {
numericLiteral(ctx, Byte.MinValue, Byte.MaxValue, ByteType.simpleString)(_.toByte)
}

/**
* Create a Short Literal expression.
*/
override def visitSmallIntLiteral(ctx: SmallIntLiteralContext): Literal = numericLiteral(ctx) {
_.toShort
override def visitSmallIntLiteral(ctx: SmallIntLiteralContext): Literal = {
numericLiteral(ctx, Short.MinValue, Short.MaxValue, ShortType.simpleString)(_.toShort)
}

/**
* Create a Long Literal expression.
*/
override def visitBigIntLiteral(ctx: BigIntLiteralContext): Literal = numericLiteral(ctx) {
_.toLong
override def visitBigIntLiteral(ctx: BigIntLiteralContext): Literal = {
numericLiteral(ctx, Long.MinValue, Long.MaxValue, LongType.simpleString)(_.toLong)
}

/**
* Create a Double Literal expression.
*/
override def visitDoubleLiteral(ctx: DoubleLiteralContext): Literal = numericLiteral(ctx) {
_.toDouble
override def visitDoubleLiteral(ctx: DoubleLiteralContext): Literal = {
numericLiteral(ctx, Double.MinValue, Double.MaxValue, DoubleType.simpleString)(_.toDouble)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -375,18 +375,21 @@ class ExpressionParserSuite extends PlanTest {

// Tiny Int Literal
assertEqual("10Y", Literal(10.toByte))
intercept("-1000Y")
intercept("-1000Y", s"does not fit in range [${Byte.MinValue}, ${Byte.MaxValue}]")

// Small Int Literal
assertEqual("10S", Literal(10.toShort))
intercept("40000S")
intercept("40000S", s"does not fit in range [${Short.MinValue}, ${Short.MaxValue}]")

// Long Int Literal
assertEqual("10L", Literal(10L))
intercept("78732472347982492793712334L")
intercept("78732472347982492793712334L",
s"does not fit in range [${Long.MinValue}, ${Long.MaxValue}]")

// Double Literal
assertEqual("10.0D", Literal(10.0D))
intercept("-1.8E308D", s"does not fit in range")
intercept("1.8E308D", s"does not fit in range")
// TODO we need to figure out if we should throw an exception here!
assertEqual("1E309", Literal(Double.PositiveInfinity))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ struct<>
-- !query 4 output
org.apache.spark.sql.catalyst.parser.ParseException

Value out of range. Value:"128" Radix:10(line 1, pos 7)
Numeric literal 128 does not fit in range [-128, 127] for type tinyint(line 1, pos 7)

== SQL ==
select 128Y
Expand Down Expand Up @@ -71,7 +71,7 @@ struct<>
-- !query 7 output
org.apache.spark.sql.catalyst.parser.ParseException

Value out of range. Value:"32768" Radix:10(line 1, pos 7)
Numeric literal 32768 does not fit in range [-32768, 32767] for type smallint(line 1, pos 7)

== SQL ==
select 32768S
Expand Down Expand Up @@ -101,7 +101,7 @@ struct<>
-- !query 10 output
org.apache.spark.sql.catalyst.parser.ParseException

For input string: "9223372036854775808"(line 1, pos 7)
Numeric literal 9223372036854775808 does not fit in range [-9223372036854775808, 9223372036854775807] for type bigint(line 1, pos 7)

== SQL ==
select 9223372036854775808L
Expand Down