You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by li...@apache.org on 2018/02/13 17:50:06 UTC

spark git commit: [SPARK-23316][SQL] AnalysisException after max iteration reached for IN query

Repository: spark
Updated Branches:
  refs/heads/master 263531466 -> 05d051293


[SPARK-23316][SQL] AnalysisException after max iteration reached for IN query

## What changes were proposed in this pull request?
Added flag ignoreNullability to DataType.equalsStructurally.
The previous semantic is for ignoreNullability=false.
When ignoreNullability=true equalsStructurally ignores nullability of contained types (map key types, value types, array element types, structure field types).
In.checkInputTypes calls equalsStructurally to check if the children types match. They should match regardless of nullability (which is just a hint), so it is now called with ignoreNullability=true.

## How was this patch tested?
New test in SubquerySuite

Author: Bogdan Raducanu <bo...@databricks.com>

Closes #20548 from bogdanrdc/SPARK-23316.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/05d05129
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/05d05129
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/05d05129

Branch: refs/heads/master
Commit: 05d051293fe46938e9cb012342fea6e8a3715cd4
Parents: 2635314
Author: Bogdan Raducanu <bo...@databricks.com>
Authored: Tue Feb 13 09:49:52 2018 -0800
Committer: gatorsmile <ga...@gmail.com>
Committed: Tue Feb 13 09:49:52 2018 -0800

----------------------------------------------------------------------
 .../sql/catalyst/expressions/predicates.scala     |  3 ++-
 .../org/apache/spark/sql/types/DataType.scala     | 18 ++++++++++++------
 .../org/apache/spark/sql/SubquerySuite.scala      |  5 +++++
 3 files changed, 19 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/05d05129/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index b469f5c..a6d41ea 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -157,7 +157,8 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate {
   require(list != null, "list should not be null")
 
   override def checkInputDataTypes(): TypeCheckResult = {
-    val mismatchOpt = list.find(l => !DataType.equalsStructurally(l.dataType, value.dataType))
+    val mismatchOpt = list.find(l => !DataType.equalsStructurally(l.dataType, value.dataType,
+      ignoreNullability = true))
     if (mismatchOpt.isDefined) {
       list match {
         case ListQuery(_, _, _, childOutputs) :: Nil =>

http://git-wip-us.apache.org/repos/asf/spark/blob/05d05129/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
index d6e0df1..0bef116 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
@@ -295,25 +295,31 @@ object DataType {
   }
 
   /**
-   * Returns true if the two data types share the same "shape", i.e. the types (including
-   * nullability) are the same, but the field names don't need to be the same.
+   * Returns true if the two data types share the same "shape", i.e. the types
+   * are the same, but the field names don't need to be the same.
+   *
+   * @param ignoreNullability whether to ignore nullability when comparing the types
    */
-  def equalsStructurally(from: DataType, to: DataType): Boolean = {
+  def equalsStructurally(
+      from: DataType,
+      to: DataType,
+      ignoreNullability: Boolean = false): Boolean = {
     (from, to) match {
       case (left: ArrayType, right: ArrayType) =>
         equalsStructurally(left.elementType, right.elementType) &&
-          left.containsNull == right.containsNull
+          (ignoreNullability || left.containsNull == right.containsNull)
 
       case (left: MapType, right: MapType) =>
         equalsStructurally(left.keyType, right.keyType) &&
           equalsStructurally(left.valueType, right.valueType) &&
-          left.valueContainsNull == right.valueContainsNull
+          (ignoreNullability || left.valueContainsNull == right.valueContainsNull)
 
       case (StructType(fromFields), StructType(toFields)) =>
         fromFields.length == toFields.length &&
           fromFields.zip(toFields)
             .forall { case (l, r) =>
-              equalsStructurally(l.dataType, r.dataType) && l.nullable == r.nullable
+              equalsStructurally(l.dataType, r.dataType) &&
+                (ignoreNullability || l.nullable == r.nullable)
             }
 
       case (fromDataType, toDataType) => fromDataType == toDataType

http://git-wip-us.apache.org/repos/asf/spark/blob/05d05129/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
index 8673dc1..31e8b0e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
@@ -950,4 +950,9 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
     assert(join.duplicateResolved)
     assert(optimizedPlan.resolved)
   }
+
+  test("SPARK-23316: AnalysisException after max iteration reached for IN query") {
+    // before the fix this would throw AnalysisException
+    spark.range(10).where("(id,id) in (select id, null from range(3))").count
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org