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 @@ -191,21 +191,6 @@ trait CheckAnalysis extends PredicateHelper {
s"join condition '${condition.sql}' " +
s"of type ${condition.dataType.simpleString} is not a boolean.")

case j @ Join(_, _, _, Some(condition)) =>
def checkValidJoinConditionExprs(expr: Expression): Unit = expr match {
case p: Predicate =>
p.asInstanceOf[Expression].children.foreach(checkValidJoinConditionExprs)
case e if e.dataType.isInstanceOf[BinaryType] =>
failAnalysis(s"binary type expression ${e.sql} cannot be used " +
"in join conditions")
case e if e.dataType.isInstanceOf[MapType] =>
failAnalysis(s"map type expression ${e.sql} cannot be used " +
"in join conditions")
case _ => // OK
}

checkValidJoinConditionExprs(condition)

case Aggregate(groupingExprs, aggregateExprs, child) =>
def checkValidAggregateExpression(expr: Expression): Unit = expr match {
case aggExpr: AggregateExpression =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -415,6 +415,21 @@ case class EqualTo(left: Expression, right: Expression)

override def inputType: AbstractDataType = AnyDataType

override def checkInputDataTypes(): TypeCheckResult = {
super.checkInputDataTypes() match {
case TypeCheckResult.TypeCheckSuccess =>
// TODO: although map type is not orderable, technically map type should be able to be used
// in equality comparison, remove this type check once we support it.
if (left.dataType.existsRecursively(_.isInstanceOf[MapType])) {
TypeCheckResult.TypeCheckFailure("Cannot use map type in EqualTo, but the actual " +
s"input type is ${left.dataType.catalogString}.")
} else {
TypeCheckResult.TypeCheckSuccess
}
case failure => failure
}
}

override def symbol: String = "="

protected override def nullSafeEval(input1: Any, input2: Any): Any = {
Expand All @@ -441,6 +456,21 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp

override def inputType: AbstractDataType = AnyDataType

override def checkInputDataTypes(): TypeCheckResult = {
super.checkInputDataTypes() match {
case TypeCheckResult.TypeCheckSuccess =>
// TODO: although map type is not orderable, technically map type should be able to be used
// in equality comparison, remove this type check once we support it.
if (left.dataType.existsRecursively(_.isInstanceOf[MapType])) {
TypeCheckResult.TypeCheckFailure("Cannot use map type in EqualNullSafe, but the actual " +
s"input type is ${left.dataType.catalogString}.")
} else {
TypeCheckResult.TypeCheckSuccess
}
case failure => failure
}
}

override def symbol: String = "<=>"

override def nullable: Boolean = false
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -451,34 +451,22 @@ class AnalysisErrorSuite extends AnalysisTest {
"another aggregate function." :: Nil)
}

test("Join can't work on binary and map types") {
val plan =
Join(
LocalRelation(
AttributeReference("a", BinaryType)(exprId = ExprId(2)),
AttributeReference("b", IntegerType)(exprId = ExprId(1))),
LocalRelation(
AttributeReference("c", BinaryType)(exprId = ExprId(4)),
AttributeReference("d", IntegerType)(exprId = ExprId(3))),
Inner,
Some(EqualTo(AttributeReference("a", BinaryType)(exprId = ExprId(2)),
AttributeReference("c", BinaryType)(exprId = ExprId(4)))))

assertAnalysisError(plan, "binary type expression `a` cannot be used in join conditions" :: Nil)

val plan2 =
Join(
LocalRelation(
AttributeReference("a", MapType(IntegerType, StringType))(exprId = ExprId(2)),
AttributeReference("b", IntegerType)(exprId = ExprId(1))),
LocalRelation(
AttributeReference("c", MapType(IntegerType, StringType))(exprId = ExprId(4)),
AttributeReference("d", IntegerType)(exprId = ExprId(3))),
Inner,
Some(EqualTo(AttributeReference("a", MapType(IntegerType, StringType))(exprId = ExprId(2)),
AttributeReference("c", MapType(IntegerType, StringType))(exprId = ExprId(4)))))

assertAnalysisError(plan2, "map type expression `a` cannot be used in join conditions" :: Nil)
test("Join can work on binary types but can't work on map types") {
val left = LocalRelation('a.binary, 'b.map(StringType, StringType))
val right = LocalRelation('c.binary, 'd.map(StringType, StringType))

val plan1 = left.join(
right,
joinType = Inner,
condition = Some('a === 'c))

assertAnalysisSuccess(plan1)

val plan2 = left.join(
right,
joinType = Inner,
condition = Some('b === 'd))
assertAnalysisError(plan2, "Cannot use map type in EqualTo" :: Nil)
}

test("PredicateSubQuery is used outside of a filter") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,8 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite {
assertErrorForDifferingTypes(GreaterThan('intField, 'booleanField))
assertErrorForDifferingTypes(GreaterThanOrEqual('intField, 'booleanField))

assertError(EqualTo('mapField, 'mapField), "Cannot use map type in EqualTo")
assertError(EqualNullSafe('mapField, 'mapField), "Cannot use map type in EqualNullSafe")
assertError(LessThan('mapField, 'mapField),
s"requires ${TypeCollection.Ordered.simpleString} type")
assertError(LessThanOrEqual('mapField, 'mapField),
Expand Down