Skip to content

Commit

Permalink
[SPARK-23774][SQL] Cast to CHAR/VARCHAR should truncate the values
Browse files Browse the repository at this point in the history
  • Loading branch information
dongjoon-hyun committed May 21, 2018
1 parent a33dcf4 commit f19cda3
Show file tree
Hide file tree
Showing 2 changed files with 49 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -1173,7 +1173,18 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
* Create a [[Cast]] expression.
*/
override def visitCast(ctx: CastContext): Expression = withOrigin(ctx) {
Cast(expression(ctx.expression), visitSparkDataType(ctx.dataType))
typedVisit[DataType](ctx.dataType) match {
case t: CharType =>
validate(t.length > 0, s"Char length ${t.length} is out of range [1, 255]", ctx)
Substring(Cast(expression(ctx.expression), visitSparkDataType(ctx.dataType)),
Literal(1), Literal(t.length))
case t: VarcharType =>
validate(t.length > 0, s"VarChar length ${t.length} is out of range [1, 65535]", ctx)
Substring(Cast(expression(ctx.expression), visitSparkDataType(ctx.dataType)),
Literal(1), Literal(t.length))
case _ =>
Cast(expression(ctx.expression), visitSparkDataType(ctx.dataType))
}
}

/**
Expand Down
37 changes: 37 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean

import org.apache.spark.{AccumulatorSuite, SparkException}
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.util.StringUtils
import org.apache.spark.sql.execution.aggregate
import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec}
Expand Down Expand Up @@ -2792,4 +2793,40 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
}
}
}

test("`Cast` to CHAR/VARCHAR should truncate the values") {
withTable("t") {
val m = intercept[ParseException] {
sql("SELECT CAST('abc' AS CHAR(0))")
}.getMessage
assert(m.contains("Char length 0 is out of range [1, 255]"))

val m2 = intercept[ParseException] {
sql("SELECT CAST('abc' AS VARCHAR(0))")
}.getMessage
assert(m2.contains("VarChar length 0 is out of range [1, 65535]"))

checkAnswer(
sql("SELECT CAST('abc' AS CHAR(2)), CAST('abc' AS CHAR(4))"),
Row("ab", "abc"))

sql("CREATE TABLE t(a STRING) USING PARQUET")
sql("INSERT INTO t VALUES ('abc')")
sql("INSERT INTO t VALUES (null)")

checkAnswer(
sql("SELECT CAST(a AS CHAR(2)), CAST(a AS CHAR(3)), CAST(a AS CHAR(4)) FROM t"),
Row("ab", "abc", "abc") :: Row(null, null, null) :: Nil)

sql(
"""
|CREATE TABLE t_ctas
|USING ORC
|AS SELECT CAST(a AS CHAR(2)) c1, CAST(a AS CHAR(3)) c2, CAST(a AS CHAR(4)) c3 FROM t
""".stripMargin)
checkAnswer(
sql("SELECT * FROM t_ctas"),
Row("ab", "abc", "abc") :: Row(null, null, null) :: Nil)
}
}
}

0 comments on commit f19cda3

Please sign in to comment.