Skip to content

Commit

Permalink
using AnalysisException
Browse files Browse the repository at this point in the history
  • Loading branch information
scwf committed Feb 11, 2015
1 parent f6b0d71 commit db86dc4
Show file tree
Hide file tree
Showing 3 changed files with 5 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -84,9 +84,8 @@ class Analyzer(catalog: Catalog,
def apply(plan: LogicalPlan): LogicalPlan = plan.transform {
case q: LogicalPlan =>
q transformExpressions {
case cast @ Cast(child, dataType) if cast.childrenResolved && !cast.resolved =>
throw new TreeNodeException(q,
s"can not cast from ${child.dataType} to $dataType!")
case cast @ Cast(child, dataType) if !cast.resolve(child.dataType, dataType) =>
throw new AnalysisException(s"can not cast from ${child.dataType} to $dataType!")
case p => p
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w

private[this] def resolvableNullability(from: Boolean, to: Boolean) = !from || to

private[this] def resolve(from: DataType, to: DataType): Boolean = {
private[sql] def resolve(from: DataType, to: DataType): Boolean = {
(from, to) match {
case (from, to) if from == to => true

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,7 @@ import scala.util.Try
import org.apache.hadoop.hive.conf.HiveConf.ConfVars

import org.apache.spark.{SparkFiles, SparkException}
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions.Cast
import org.apache.spark.sql.{DataFrame, Row}
import org.apache.spark.sql.{AnalysisException, DataFrame, Row}
import org.apache.spark.sql.catalyst.plans.logical.Project
import org.apache.spark.sql.Dsl._
import org.apache.spark.sql.hive._
Expand Down Expand Up @@ -69,7 +67,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
}

test("SPARK-5649: added a rule to check datatypes cast") {
intercept[TreeNodeException[Cast]] {
intercept[AnalysisException] {
sql("select cast(key as binary) from src").collect()
}
}
Expand Down

0 comments on commit db86dc4

Please sign in to comment.