From dfe501a087b4292f7327ad04bb0742eeedffab9f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 10 Jan 2016 00:57:12 -0800 Subject: [PATCH 01/10] parse subquery --- .../sql/catalyst/parser/ExpressionParser.g | 2 + .../spark/sql/catalyst/CatalystQl.scala | 9 +++ .../sql/catalyst/analysis/Analyzer.scala | 21 +++++++ .../sql/catalyst/expressions/subquery.scala | 60 +++++++++++++++++++ .../spark/sql/catalyst/CatalystQlSuite.scala | 20 +++++++ 5 files changed, 112 insertions(+) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g index aabb5d49582c8..12414b4d5111a 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g @@ -197,6 +197,8 @@ atomExpression | whenExpression | (functionName LPAREN) => function | tableOrColumn + | (LPAREN KW_SELECT) => subQueryExpression + -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP) subQueryExpression) | LPAREN! expression RPAREN! ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index 1eda4a9a97644..ecd2476c90484 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -619,6 +619,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val BETWEEN = "(?i)BETWEEN".r val WHEN = "(?i)WHEN".r val CASE = "(?i)CASE".r + val EXISTS = "(?i)EXISTS".r protected def nodeToExpr(node: ASTNode): Expression = node match { /* Attribute References */ @@ -630,6 +631,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C UnresolvedAttribute(nameParts :+ cleanIdentifier(attr)) case other => UnresolvedExtractValue(other, Literal(attr)) } + case Token("TOK_SUBQUERY_EXPR", Token("TOK_SUBQUERY_OP", Nil) :: subquery :: Nil) => + ScalarSubQuery(nodeToPlan(subquery)) /* Stars (*) */ case Token("TOK_ALLCOLREF", Nil) => UnresolvedStar(None) @@ -712,6 +715,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C IsNull(nodeToExpr(child)) case Token("TOK_FUNCTION", Token(IN(), Nil) :: value :: list) => In(nodeToExpr(value), list.map(nodeToExpr)) + case Token("TOK_SUBQUERY_EXPR", + Token("TOK_SUBQUERY_OP", Token(IN(), Nil) :: Nil) :: subquery :: value :: Nil) => + InSubQuery(nodeToExpr(value), nodeToPlan(subquery)) case Token("TOK_FUNCTION", Token(BETWEEN(), Nil) :: kw :: @@ -734,6 +740,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) case Token(NOT(), child :: Nil) => Not(nodeToExpr(child)) case Token("!", child :: Nil) => Not(nodeToExpr(child)) + case Token("TOK_SUBQUERY_EXPR", + Token("TOK_SUBQUERY_OP", Token(EXISTS(), Nil) :: Nil) :: subquery :: Nil) => + Exists(nodeToPlan(subquery)) /* Case statements */ case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8a33af8207350..6b1f3efa0ef8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -74,6 +74,7 @@ class Analyzer( ResolvePivot :: ResolveUpCast :: ResolveSortReferences :: + ResolveSubquery :: ResolveGenerate :: ResolveFunctions :: ResolveAliases :: @@ -590,6 +591,26 @@ class Analyzer( } } + object ResolveSubquery extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case q: LogicalPlan => + q transformExpressions { + case e: SubQueryExpression if e.query.resolved => + val afterRule = execute(e.query) + if (afterRule.resolved) { + e match { + case InSubQuery(value, _) => InSubQuery(value, afterRule) + case ScalarSubQuery(_) => ScalarSubQuery(afterRule) + case Exists(_) => Exists(afterRule) + } + } else { + // TODO: rewrite as join + e + } + } + } + } + /** * Turns projections that contain aggregate expressions into aggregations. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala new file mode 100644 index 0000000000000..8efc72ac6d642 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types.{BooleanType, DataType, NullType} + + +abstract class SubQueryExpression extends LeafExpression { + def query: LogicalPlan +} + +case class ScalarSubQuery(query: LogicalPlan) extends SubQueryExpression with CodegenFallback { + override lazy val resolved: Boolean = query.resolved + override def dataType: DataType = { + if (resolved) { + query.schema.fields(0).dataType + } else { + NullType + } + } + override def nullable: Boolean = true + override def foldable: Boolean = true + + private lazy val result: Any = { + // SQLContext + } + override def eval(input: InternalRow): Any = { + + } +} + +case class InSubQuery(value: Expression, query: LogicalPlan) + extends SubQueryExpression with Unevaluable { + override def dataType: DataType = BooleanType + override def nullable: Boolean = false +} + +case class Exists(query: LogicalPlan) extends SubQueryExpression with Unevaluable { + override def dataType: DataType = BooleanType + override def nullable: Boolean = false + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala index d7204c3488313..f9be90e2e3e60 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala @@ -49,4 +49,24 @@ class CatalystQlSuite extends PlanTest { parser.createPlan("select sum(product + 1) over (partition by (product + (1)) order by 2) " + "from windowData") } + + test("subquery") { + parser.createPlan("select (select max(b) from s) ss from t") + + parser.createPlan("select * from t where a = (select b from s)") + parser.createPlan("select * from t where a > (select b from s)") + parser.createPlan("select * from t where a in (select b from s)") + parser.createPlan("select * from t where a not in (select b from s)") + parser.createPlan("select * from t where a and exists (select b from s)") + parser.createPlan("select * from t where not exists (select b from s)") + parser.createPlan("select * from t where exists (select b from s)") + + parser.createPlan("select * from t group by g having a = (select b from s)") + parser.createPlan("select * from t group by g having a > (select b from s)") + parser.createPlan("select * from t group by g having a in (select b from s)") + parser.createPlan("select * from t group by g having a not in (select b from s)") + parser.createPlan("select * from t group by g having exists (select b from s)") + parser.createPlan("select * from t group by g having not exists (select b from s)") + parser.createPlan("select * from t group by g having exists (select b from s)") + } } From b855816c6bab14d0166f6d2f9d19044c194a1f96 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 10 Jan 2016 01:36:04 -0800 Subject: [PATCH 02/10] add left anti join --- .../sql/catalyst/optimizer/Optimizer.scala | 6 ++-- .../spark/sql/catalyst/plans/joinTypes.scala | 5 ++++ .../plans/logical/basicOperators.scala | 2 +- .../org/apache/spark/sql/DataFrame.scala | 10 +++---- .../spark/sql/execution/SparkStrategies.scala | 17 ++++++----- ...JoinHash.scala => BroadcastSemiJoin.scala} | 4 ++- .../sql/execution/joins/HashSemiJoin.scala | 8 +++-- .../sql/execution/joins/LeftSemiJoinBNL.scala | 11 +++++-- .../execution/joins/LeftSemiJoinHash.scala | 2 ++ .../apache/spark/sql/DataFrameJoinSuite.scala | 13 ++++++++ .../org/apache/spark/sql/JoinSuite.scala | 4 +-- .../execution/joins/BroadcastJoinSuite.scala | 2 +- .../sql/execution/joins/SemiJoinSuite.scala | 30 +++++++++++++++---- .../spark/sql/hive/StatisticsSuite.scala | 4 +-- 14 files changed, 82 insertions(+), 36 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/joins/{BroadcastLeftSemiJoinHash.scala => BroadcastSemiJoin.scala} (96%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index f8121a733a8d2..a95cc8f5cb3e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, EliminateSubQueri import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.planning.ExtractFiltersAndInnerJoins -import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftOuter, LeftSemi, RightOuter} +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.types._ @@ -854,7 +854,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { (leftFilterConditions ++ commonFilterCondition). reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin) - case _ @ (LeftOuter | LeftSemi) => + case _ @ (LeftOuter | LeftSemi | LeftAnti) => // push down the left side only `where` condition val newLeft = leftFilterConditions. reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) @@ -873,7 +873,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right) joinType match { - case _ @ (Inner | LeftSemi) => + case _ @ (Inner | LeftSemi | LeftAnti) => // push down the single side only join filter for both sides sub queries val newLeft = leftJoinConditions. reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index a5f6764aef7ce..1695a22386748 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -24,6 +24,7 @@ object JoinType { case "leftouter" | "left" => LeftOuter case "rightouter" | "right" => RightOuter case "leftsemi" => LeftSemi + case "leftanti" => LeftAnti case _ => val supported = Seq( "inner", @@ -60,3 +61,7 @@ case object FullOuter extends JoinType { case object LeftSemi extends JoinType { override def sql: String = "LEFT SEMI" } + +case object LeftAnti extends JoinType { + override def sql: String = "LEFT ANTI" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 64957db6b4013..dee95c79e0c97 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -131,7 +131,7 @@ case class Join( override def output: Seq[Attribute] = { joinType match { - case LeftSemi => + case LeftSemi | LeftAnti => left.output case LeftOuter => left.output ++ right.output.map(_.withNullability(true)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 60d2f05b8605b..918192debd5dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql import java.io.CharArrayWriter -import java.util.Properties import scala.language.implicitConversions import scala.reflect.ClassTag @@ -30,16 +29,15 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.PythonRDD import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection, SqlParser} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, FileRelation, LogicalRDD, Queryable, QueryExecution, SQLExecution} -import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection, SqlParser} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator -import org.apache.spark.sql.sources.HadoopFsRelation +import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation} +import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, FileRelation, LogicalRDD, QueryExecution, Queryable, SQLExecution} import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -466,7 +464,7 @@ class DataFrame private[sql]( // Project only one of the join columns. val joinedCols = JoinType(joinType) match { - case Inner | LeftOuter | LeftSemi => + case Inner | LeftOuter | LeftSemi | LeftAnti => usingColumns.map(col => withPlan(joined.left).resolve(col)) case RightOuter => usingColumns.map(col => withPlan(joined.right).resolve(col)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 482130a18d939..4fdab3e7c7cd4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -34,17 +34,18 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object LeftSemiJoin extends Strategy with PredicateHelper { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ExtractEquiJoinKeys( - LeftSemi, leftKeys, rightKeys, condition, left, CanBroadcast(right)) => - joins.BroadcastLeftSemiJoinHash( - leftKeys, rightKeys, planLater(left), planLater(right), condition) :: Nil + case ExtractEquiJoinKeys(joinType @ (LeftSemi | LeftAnti), + leftKeys, rightKeys, condition, left, CanBroadcast(right)) => + joins.BroadcastSemiJoin( + joinType, leftKeys, rightKeys, planLater(left), planLater(right), condition) :: Nil // Find left semi joins where at least some predicates can be evaluated by matching join keys - case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) => + case ExtractEquiJoinKeys( + joinType @ (LeftSemi | LeftAnti), leftKeys, rightKeys, condition, left, right) => joins.LeftSemiJoinHash( - leftKeys, rightKeys, planLater(left), planLater(right), condition) :: Nil + joinType, leftKeys, rightKeys, planLater(left), planLater(right), condition) :: Nil // no predicate can be evaluated by matching hash keys - case logical.Join(left, right, LeftSemi, condition) => - joins.LeftSemiJoinBNL(planLater(left), planLater(right), condition) :: Nil + case logical.Join(left, right, joinType @ (LeftSemi | LeftAnti), condition) => + joins.LeftSemiJoinBNL(joinType, planLater(left), planLater(right), condition) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala index 004407b2e6925..77d12971edae3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.{InternalAccumulator, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -28,7 +29,8 @@ import org.apache.spark.sql.execution.metric.SQLMetrics * Build the right table's join keys into a HashSet, and iteratively go through the left * table, to find the if join keys are in the Hash set. */ -case class BroadcastLeftSemiJoinHash( +case class BroadcastSemiJoin( + joinType: JoinType, leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SparkPlan, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala index 3e0f74cd98c21..da778d742301d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala @@ -19,12 +19,14 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.{LeftSemi, JoinType} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.LongSQLMetric trait HashSemiJoin { self: SparkPlan => + val joinType: JoinType val leftKeys: Seq[Expression] val rightKeys: Seq[Expression] val left: SparkPlan @@ -72,7 +74,7 @@ trait HashSemiJoin { streamIter.filter(current => { numStreamRows += 1 val key = joinKeys(current) - val r = !key.anyNull && hashSet.contains(key) + val r = (joinType == LeftSemi) == (!key.anyNull && hashSet.contains(key)) if (r) numOutputRows += 1 r }) @@ -89,9 +91,9 @@ trait HashSemiJoin { numStreamRows += 1 val key = joinKeys(current) lazy val rowBuffer = hashedRelation.get(key) - val r = !key.anyNull && rowBuffer != null && rowBuffer.exists { + val r = (joinType == LeftSemi) == (!key.anyNull && rowBuffer != null && rowBuffer.exists { (row: InternalRow) => boundCondition(joinedRow(current, row)) - } + }) if (r) numOutputRows += 1 r } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index 82498ee395649..f1958654f525d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.{LeftSemi, JoinType} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -29,7 +30,10 @@ import org.apache.spark.sql.execution.metric.SQLMetrics * for hash join. */ case class LeftSemiJoinBNL( - streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression]) + joinType: JoinType, + streamed: SparkPlan, + broadcast: SparkPlan, + condition: Option[Expression]) extends BinaryNode { // TODO: Override requiredChildDistribution. @@ -77,10 +81,11 @@ case class LeftSemiJoinBNL( } i += 1 } - if (matched) { + val r = (joinType == LeftSemi) == matched + if (r) { numOutputRows += 1 } - matched + r }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala index 25b3b5ca2377f..80f29d585c1c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -29,6 +30,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics * table, to find the if join keys are in the Hash set. */ case class LeftSemiJoinHash( + joinType: JoinType, leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SparkPlan, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 39a65413bd592..1263b1236b81f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -33,6 +33,19 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { Row(1, "1", "2") :: Row(2, "2", "3") :: Row(3, "3", "4") :: Nil) } + test("join - semi/anti join") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + val df2 = Seq(3, 4, 5).map(i => (i, (i + 1).toString)).toDF("int", "str") + + checkAnswer( + df.join(df2, Seq("int"), "left_semi"), + Row(3, "3") :: Nil) + + checkAnswer( + df.join(df2, Seq("int"), "left_anti"), + Row(1, "1") :: Row(2, "2") :: Nil) + } + test("join - join using multiple columns") { val df = Seq(1, 2, 3).map(i => (i, i + 1, i.toString)).toDF("int", "int2", "str") val df2 = Seq(1, 2, 3).map(i => (i, i + 1, (i + 1).toString)).toDF("int", "int2", "str") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 9a3c262e9485d..6233759723a9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -50,7 +50,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { case j: LeftSemiJoinBNL => j case j: CartesianProduct => j case j: BroadcastNestedLoopJoin => j - case j: BroadcastLeftSemiJoinHash => j + case j: BroadcastSemiJoin => j case j: SortMergeJoin => j case j: SortMergeOuterJoin => j } @@ -423,7 +423,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000000000") { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", - classOf[BroadcastLeftSemiJoinHash]) + classOf[BroadcastSemiJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index 58581d71e1bc1..26328a4f52473 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -77,7 +77,7 @@ class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { } test("unsafe broadcast left semi join updates peak execution memory") { - testBroadcastJoin[BroadcastLeftSemiJoinHash]("unsafe broadcast left semi join", "leftsemi") + testBroadcastJoin[BroadcastSemiJoin]("unsafe broadcast left semi join", "leftsemi") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala index 9c86084f9b8a9..d734457a89c0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.{DataFrame, Row, SQLConf} import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys -import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.{JoinType, LeftAnti, LeftSemi} import org.apache.spark.sql.execution.{EnsureRequirements, SparkPlan, SparkPlanTest} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} +import org.apache.spark.sql.{DataFrame, Row, SQLConf} class SemiJoinSuite extends SparkPlanTest with SharedSQLContext { @@ -63,10 +63,11 @@ class SemiJoinSuite extends SparkPlanTest with SharedSQLContext { leftRows: => DataFrame, rightRows: => DataFrame, condition: => Expression, + joinType: JoinType, expectedAnswer: Seq[Product]): Unit = { def extractJoinParts(): Option[ExtractEquiJoinKeys.ReturnType] = { - val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, Inner, Some(condition)) + val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, joinType, Some(condition)) ExtractEquiJoinKeys.unapply(join) } @@ -75,7 +76,7 @@ class SemiJoinSuite extends SparkPlanTest with SharedSQLContext { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext).apply( - LeftSemiJoinHash(leftKeys, rightKeys, left, right, boundCondition)), + LeftSemiJoinHash(joinType, leftKeys, rightKeys, left, right, boundCondition)), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } @@ -86,7 +87,7 @@ class SemiJoinSuite extends SparkPlanTest with SharedSQLContext { extractJoinParts().foreach { case (joinType, leftKeys, rightKeys, boundCondition, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - BroadcastLeftSemiJoinHash(leftKeys, rightKeys, left, right, boundCondition), + BroadcastSemiJoin(joinType, leftKeys, rightKeys, left, right, boundCondition), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } @@ -96,7 +97,7 @@ class SemiJoinSuite extends SparkPlanTest with SharedSQLContext { test(s"$testName using LeftSemiJoinBNL") { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - LeftSemiJoinBNL(left, right, Some(condition)), + LeftSemiJoinBNL(joinType, left, right, Some(condition)), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } @@ -108,9 +109,26 @@ class SemiJoinSuite extends SparkPlanTest with SharedSQLContext { left, right, condition, + LeftSemi, Seq( (2, 1.0), (2, 1.0) ) ) + + testLeftSemiJoin( + "anti join test", + left, + right, + condition, + LeftAnti, + Seq( + (1, 2.0), + (1, 2.0), + (3, 3.0), + (null, null), + (null, 5.0), + (6, null) + ) + ) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 78f74cdc19ddb..23d58801feafc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -207,7 +207,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. var bhj = df.queryExecution.sparkPlan.collect { - case j: BroadcastLeftSemiJoinHash => j + case j: BroadcastSemiJoin => j } assert(bhj.size === 1, s"actual query plans do not contain broadcast join: ${df.queryExecution}") @@ -220,7 +220,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1") df = sql(leftSemiJoinQuery) bhj = df.queryExecution.sparkPlan.collect { - case j: BroadcastLeftSemiJoinHash => j + case j: BroadcastSemiJoin => j } assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") From 9918c0cc14077e803ec73a42c75a62bb0383e456 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 10 Jan 2016 15:56:36 -0800 Subject: [PATCH 03/10] support exists / not exists --- .../sql/catalyst/analysis/Analyzer.scala | 68 ++++++++++++++++--- .../sql/catalyst/expressions/subquery.scala | 3 +- .../sql/hive/execution/HiveQuerySuite.scala | 11 +++ 3 files changed, 70 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6b1f3efa0ef8c..37556a3218b07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -591,23 +592,68 @@ class Analyzer( } } - object ResolveSubquery extends Rule[LogicalPlan] { + object ResolveSubquery extends Rule[LogicalPlan] with PredicateHelper { + + def hasSubquery(e: Expression): Boolean = { + e.find(_.isInstanceOf[SubQueryExpression]).isDefined + } + + def hasSubquery(q: LogicalPlan): Boolean = { + q.expressions.exists(hasSubquery) + } + + def removeUnresolvedFilter(q: LogicalPlan): (LogicalPlan, Option[Expression]) = { + val unresolvedConditions = ArrayBuffer[Expression]() + val removed = q transformUp { + case f @ Filter(cond, child) => + val (resolved, unresolved) = splitConjunctivePredicates(cond).partition(_.resolved) + unresolvedConditions ++= unresolved + if (resolved.nonEmpty) { + Filter(resolved.reduceLeft(And), child) + } else { + child + } + } + (removed, unresolvedConditions.reduceLeftOption(And)) + } + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case q: LogicalPlan => - q transformExpressions { - case e: SubQueryExpression if e.query.resolved => - val afterRule = execute(e.query) + case q: LogicalPlan if q.childrenResolved && hasSubquery(q) => + + var afterResolve = q transformExpressions { + case e @ ScalarSubQuery(subquery) if !subquery.resolved => + val afterRule = execute(subquery) if (afterRule.resolved) { - e match { - case InSubQuery(value, _) => InSubQuery(value, afterRule) - case ScalarSubQuery(_) => ScalarSubQuery(afterRule) - case Exists(_) => Exists(afterRule) - } + ScalarSubQuery(afterRule) } else { - // TODO: rewrite as join e } } + + afterResolve match { + case f @ Filter(condition, child) => + val (withSubquery, withoutSubquery) = + splitConjunctivePredicates(condition).partition(hasSubquery) + var newChild: LogicalPlan = child + withSubquery.foreach { + case Exists(sub) => + // use all the conditions as join condition + val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) + newChild = Join(newChild, removed, LeftSemi, joinCondition) + case Not(Exists(sub)) => + val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) + newChild = Join(newChild, removed, LeftAnti, joinCondition) + // TODO: support Or(Exists(), Exists()) + case InSubQuery(value, sub) => + case Not(InSubQuery(value, sub)) => + case _ => + } + if (withoutSubquery.nonEmpty) { + Filter(withoutSubquery.reduceLeft(And), newChild) + } else { + newChild + } + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 8efc72ac6d642..5408c16cda338 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -49,12 +49,13 @@ case class ScalarSubQuery(query: LogicalPlan) extends SubQueryExpression with Co case class InSubQuery(value: Expression, query: LogicalPlan) extends SubQueryExpression with Unevaluable { + override lazy val resolved: Boolean = false // can't be resolved override def dataType: DataType = BooleanType override def nullable: Boolean = false } case class Exists(query: LogicalPlan) extends SubQueryExpression with Unevaluable { + override lazy val resolved: Boolean = false // can't be resolved override def dataType: DataType = BooleanType override def nullable: Boolean = false - } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 4659d745fe78b..45164e87168a9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -806,6 +806,17 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } + test("exists") { + assertResult(Array(Row(1))) { + sql("select * from (select 1 as a union all select 2 as a) t " + + "where exists (select * from (select 1 as b) t2 where b = a and b < 2) ").collect() + } + assertResult(Array(Row(2))) { + sql("select * from (select 1 as a union all select 2 as a) t " + + "where not exists (select * from (select 1 as b) t2 where b = a and b < 2) ").collect() + } + } + test("SPARK-5383 alias for udfs with multi output columns") { assert( sql("select stack(2, key, value, key, value) as (a, b) from src limit 5") From 37f22e2f145dc48ae71c8bf022973abc9538a8fe Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 10 Jan 2016 22:28:06 -0800 Subject: [PATCH 04/10] support in subquery --- .../spark/sql/catalyst/CatalystQl.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 80 +++++++++++++------ .../sql/catalyst/expressions/subquery.scala | 13 ++- .../sql/hive/execution/HiveQuerySuite.scala | 13 +++ 4 files changed, 79 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index ecd2476c90484..99e263f0944d2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -717,7 +717,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C In(nodeToExpr(value), list.map(nodeToExpr)) case Token("TOK_SUBQUERY_EXPR", Token("TOK_SUBQUERY_OP", Token(IN(), Nil) :: Nil) :: subquery :: value :: Nil) => - InSubQuery(nodeToExpr(value), nodeToPlan(subquery)) + In(nodeToExpr(value), ListSubQuery(nodeToPlan(subquery)) :: Nil) case Token("TOK_FUNCTION", Token(BETWEEN(), Nil) :: kw :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 37556a3218b07..90cddec35c411 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -118,7 +118,13 @@ class Analyzer( withAlias.getOrElse(relation) } substituted.getOrElse(u) + case other => + other transformExpressions { + case e: SubQueryExpression => + e.withNewPlan(substituteCTE(e.query, cteRelations)) + } } + } } @@ -617,10 +623,24 @@ class Analyzer( (removed, unresolvedConditions.reduceLeftOption(And)) } + def rewriteInSubquery(value: Expression, subquery: LogicalPlan): (LogicalPlan, Expression) = { + val (removed, joinCondition) = removeUnresolvedFilter(execute(subquery)) + val resolved = execute(removed) + assert(resolved.resolved) + val right = resolved.output(0) + val equalCond = EqualTo(value, right) + val cond = if (joinCondition.isDefined){ + And(joinCondition.get, equalCond) + } else { + equalCond + } + (resolved, cond) + } + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case q: LogicalPlan if q.childrenResolved && hasSubquery(q) => - var afterResolve = q transformExpressions { + val afterResolve = q transformExpressions { case e @ ScalarSubQuery(subquery) if !subquery.resolved => val afterRule = execute(subquery) if (afterRule.resolved) { @@ -630,29 +650,41 @@ class Analyzer( } } - afterResolve match { - case f @ Filter(condition, child) => - val (withSubquery, withoutSubquery) = - splitConjunctivePredicates(condition).partition(hasSubquery) - var newChild: LogicalPlan = child - withSubquery.foreach { - case Exists(sub) => - // use all the conditions as join condition - val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) - newChild = Join(newChild, removed, LeftSemi, joinCondition) - case Not(Exists(sub)) => - val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) - newChild = Join(newChild, removed, LeftAnti, joinCondition) - // TODO: support Or(Exists(), Exists()) - case InSubQuery(value, sub) => - case Not(InSubQuery(value, sub)) => - case _ => - } - if (withoutSubquery.nonEmpty) { - Filter(withoutSubquery.reduceLeft(And), newChild) - } else { - newChild - } + if (afterResolve.resolved) { + afterResolve + } else { + afterResolve match { + case f @ Filter(condition, child) => + val (withSubquery, withoutSubquery) = + splitConjunctivePredicates(condition).partition(hasSubquery) + var newChild: LogicalPlan = child + withSubquery.foreach { + case Exists(sub) => + // use all the conditions as join condition + val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) + newChild = Join(newChild, removed, LeftSemi, joinCondition) + case Not(Exists(sub)) => + val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) + newChild = Join(newChild, removed, LeftAnti, joinCondition) + // TODO: support Or(Exists(), Exists()) + case In(value, ListSubQuery(sub) :: Nil) => + val (resolved, cond) = rewriteInSubquery(value, sub) + newChild = Join(newChild, resolved, LeftSemi, Some(cond)) + case Not(In(value, ListSubQuery(sub) :: Nil)) => + val (resolved, cond) = rewriteInSubquery(value, sub) + newChild = Join(newChild, resolved, LeftAnti, Some(cond)) + case other => + sys.error(s"not supported subquery: $other") + } + if (withoutSubquery.nonEmpty) { + Filter(withoutSubquery.reduceLeft(And), newChild) + } else { + newChild + } + case other => + sys.error(s"Does not support subquery in $other") + other + } } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 5408c16cda338..a1671a32e83b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -20,11 +20,12 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types.{BooleanType, DataType, NullType} +import org.apache.spark.sql.types.{ArrayType, BooleanType, DataType, NullType} abstract class SubQueryExpression extends LeafExpression { def query: LogicalPlan + def withNewPlan(plan: LogicalPlan): SubQueryExpression } case class ScalarSubQuery(query: LogicalPlan) extends SubQueryExpression with CodegenFallback { @@ -39,6 +40,8 @@ case class ScalarSubQuery(query: LogicalPlan) extends SubQueryExpression with Co override def nullable: Boolean = true override def foldable: Boolean = true + override def withNewPlan(plan: LogicalPlan): ScalarSubQuery = ScalarSubQuery(plan) + private lazy val result: Any = { // SQLContext } @@ -47,15 +50,17 @@ case class ScalarSubQuery(query: LogicalPlan) extends SubQueryExpression with Co } } -case class InSubQuery(value: Expression, query: LogicalPlan) +case class ListSubQuery(query: LogicalPlan) extends SubQueryExpression with Unevaluable { override lazy val resolved: Boolean = false // can't be resolved - override def dataType: DataType = BooleanType - override def nullable: Boolean = false + override def dataType: DataType = ArrayType(NullType) + override def nullable: Boolean = true + override def withNewPlan(plan: LogicalPlan): ListSubQuery = ListSubQuery(plan) } case class Exists(query: LogicalPlan) extends SubQueryExpression with Unevaluable { override lazy val resolved: Boolean = false // can't be resolved override def dataType: DataType = BooleanType override def nullable: Boolean = false + override def withNewPlan(plan: LogicalPlan): Exists = Exists(plan) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 45164e87168a9..8226b93120231 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -817,6 +817,19 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } + test("in subquery") { + assertResult(Array(Row(1))) { + sql("with t2 as (select 1 as b) " + + "select * from (select 1 as a union all select 2 as a) t " + + "where a in (select b as a from t2 where b < 2) ").collect() + } + assertResult(Array(Row(2))) { + sql("with t2 as (select 1 as b) " + + "select * from (select 1 as a union all select 2 as a) t " + + "where a not in (select b as a from t2 where b < 2) ").collect() + } + } + test("SPARK-5383 alias for udfs with multi output columns") { assert( sql("select stack(2, key, value, key, value) as (a, b) from src limit 5") From fde2461a159c89569269b7dc6a2f1bcbd5a9ec70 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 11 Jan 2016 00:04:04 -0800 Subject: [PATCH 05/10] support correlated scalar subquery --- .../sql/catalyst/analysis/Analyzer.scala | 63 ++++++++++++++----- .../sql/hive/execution/HiveQuerySuite.scala | 8 +++ 2 files changed, 54 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 90cddec35c411..8e25cb226b699 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -20,13 +20,13 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} -import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftOuter, LeftSemi} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef +import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} import org.apache.spark.sql.types._ /** @@ -604,6 +604,11 @@ class Analyzer( e.find(_.isInstanceOf[SubQueryExpression]).isDefined } + def onlyHasScalarSubquery(e: Expression): Boolean = { + (e.find(x => x.isInstanceOf[Exists] || x.isInstanceOf[ListSubQuery]).isEmpty + && e.find(_.isInstanceOf[ScalarSubQuery]).isDefined) + } + def hasSubquery(q: LogicalPlan): Boolean = { q.expressions.exists(hasSubquery) } @@ -627,7 +632,7 @@ class Analyzer( val (removed, joinCondition) = removeUnresolvedFilter(execute(subquery)) val resolved = execute(removed) assert(resolved.resolved) - val right = resolved.output(0) + val right = resolved.output.head val equalCond = EqualTo(value, right) val cond = if (joinCondition.isDefined){ And(joinCondition.get, equalCond) @@ -641,13 +646,8 @@ class Analyzer( case q: LogicalPlan if q.childrenResolved && hasSubquery(q) => val afterResolve = q transformExpressions { - case e @ ScalarSubQuery(subquery) if !subquery.resolved => - val afterRule = execute(subquery) - if (afterRule.resolved) { - ScalarSubQuery(afterRule) - } else { - e - } + case e: SubQueryExpression if !e.query.resolved => + e.withNewPlan(execute(e.query)) } if (afterResolve.resolved) { @@ -655,8 +655,9 @@ class Analyzer( } else { afterResolve match { case f @ Filter(condition, child) => - val (withSubquery, withoutSubquery) = + val (withSubquery, withoutSubquery: Seq[Expression]) = splitConjunctivePredicates(condition).partition(hasSubquery) + val newConds = ArrayBuffer[Expression]() var newChild: LogicalPlan = child withSubquery.foreach { case Exists(sub) => @@ -666,24 +667,52 @@ class Analyzer( case Not(Exists(sub)) => val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) newChild = Join(newChild, removed, LeftAnti, joinCondition) - // TODO: support Or(Exists(), Exists()) case In(value, ListSubQuery(sub) :: Nil) => val (resolved, cond) = rewriteInSubquery(value, sub) newChild = Join(newChild, resolved, LeftSemi, Some(cond)) case Not(In(value, ListSubQuery(sub) :: Nil)) => val (resolved, cond) = rewriteInSubquery(value, sub) newChild = Join(newChild, resolved, LeftAnti, Some(cond)) + case expr: Expression if onlyHasScalarSubquery(expr) => + val newCond = expr.transformUp { + case ScalarSubQuery(sub) if !sub.resolved => + val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) + val (internalExprs, externalJoinExprs) = + splitConjunctivePredicates(joinCondition.get). + flatMap(_.children).partition(_.resolved) + val joinAttrs: Seq[NamedExpression] = internalExprs.map { + case e: NamedExpression => e + case e => Alias(e, e.toString)() + } + val newCond = joinAttrs.zip(externalJoinExprs).map { + case (attr, expr) => EqualTo(expr, attr.toAttribute) + }.reduceLeftOption(And) + val resolved = execute(removed) + assert(resolved.resolved) + val newAgg = resolved match { + // the scalar subquery can have not grouping keys and single + // AggregateExpression + case Aggregate(Nil, aggregateExpression :: Nil, child) => + Aggregate( + internalExprs, + joinAttrs ++ Seq(aggregateExpression), + child) + } + newChild = Join(newChild, newAgg, LeftOuter, newCond) + val output = resolved.output.head + output + } + newConds += newCond case other => + // TODO: support Or(Exists(), Exists()) sys.error(s"not supported subquery: $other") } - if (withoutSubquery.nonEmpty) { - Filter(withoutSubquery.reduceLeft(And), newChild) + if (withoutSubquery.nonEmpty || newConds.nonEmpty) { + Filter((withoutSubquery ++ newConds).reduceLeft(And), newChild) } else { newChild } - case other => - sys.error(s"Does not support subquery in $other") - other + case other => other } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 8226b93120231..f2d43fd022170 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -830,6 +830,14 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } + test("scalar subquery") { + assertResult(Array(Row(1))) { + sql("with t2 as (select 1 as b, 2 as c) " + + "select a from (select 1 as a union all select 2 as a) t " + + "where a = (select max(b) from t2 where c = a + 1) ").collect() + } + } + test("SPARK-5383 alias for udfs with multi output columns") { assert( sql("select stack(2, key, value, key, value) as (a, b) from src limit 5") From 7ae30e1ba8bdc966dbb3021c8bc2760fe3690635 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 11 Jan 2016 13:37:17 -0800 Subject: [PATCH 06/10] support scalar subquery --- .../sql/catalyst/expressions/subquery.scala | 25 ++++++++----- .../spark/sql/execution/SparkPlan.scala | 36 ++++++++++++++++++- .../sql/hive/execution/HiveQuerySuite.scala | 14 +++++++- 3 files changed, 64 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index a1671a32e83b2..9d81dfcc502cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types.{ArrayType, BooleanType, DataType, NullType} @@ -30,24 +31,30 @@ abstract class SubQueryExpression extends LeafExpression { case class ScalarSubQuery(query: LogicalPlan) extends SubQueryExpression with CodegenFallback { override lazy val resolved: Boolean = query.resolved - override def dataType: DataType = { - if (resolved) { - query.schema.fields(0).dataType + + override def dataType: DataType = query.schema.fields.head.dataType + + override def checkInputDataTypes(): TypeCheckResult = { + if (query.schema.length != 1) { + TypeCheckResult.TypeCheckFailure("Scalar subquery can only have 1 column, but got " + + query.schema.length.toString) } else { - NullType + TypeCheckResult.TypeCheckSuccess } } + override def nullable: Boolean = true - override def foldable: Boolean = true + override def foldable: Boolean = false override def withNewPlan(plan: LogicalPlan): ScalarSubQuery = ScalarSubQuery(plan) - private lazy val result: Any = { - // SQLContext - } - override def eval(input: InternalRow): Any = { + private var result: Any = null + def updateResult(v: Any): Unit = { + result = v } + + override def eval(input: InternalRow): Any = result } case class ListSubQuery(query: LogicalPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 2355de3d05865..833542e496a27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -20,10 +20,12 @@ package org.apache.spark.sql.execution import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, Future, ExecutionContext} import org.apache.spark.Logging import org.apache.spark.rdd.{RDD, RDDOperationScope} -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ @@ -31,6 +33,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetric} import org.apache.spark.sql.types.DataType +import org.apache.spark.util.ThreadUtils /** * The base class for physical operators. @@ -116,7 +119,33 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ final def prepare(): Unit = { if (prepareCalled.compareAndSet(false, true)) { doPrepare() + + // collect all the subqueries and submit jobs to execute them in background + val queryResults = ArrayBuffer[(ScalarSubQuery, Future[Array[InternalRow]])]() + val allSubqueries = expressions.flatMap(_.collect {case e: ScalarSubQuery => e}) + allSubqueries.foreach { e => + val futureResult = scala.concurrent.future { + val df = DataFrame(sqlContext, e.query) + df.queryExecution.toRdd.collect() + }(SparkPlan.subqueryExecutionContext) + queryResults += e -> futureResult + } + children.foreach(_.prepare()) + + // fill in the result of subqueries + queryResults.foreach { + case (e, futureResult) => + val rows = Await.result(futureResult, Duration.Inf) + if (rows.length > 1) { + sys.error(s"Scalar subquery should return at most one row, but got ${rows.length}: " + + s"${e.query.treeString}") + } + // Analyzer will make sure that it only return on column + if (rows.length > 0) { + e.updateResult(rows(0).get(0, e.dataType)) + } + } } } @@ -253,6 +282,11 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ } } +object SparkPlan { + private[execution] val subqueryExecutionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("subquery", 16)) +} + private[sql] trait LeafNode extends SparkPlan { override def children: Seq[SparkPlan] = Nil override def producedAttributes: AttributeSet = outputSet diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index f2d43fd022170..fa60225125fcb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -830,7 +830,19 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } - test("scalar subquery") { + test("uncorrelated scalar subquery") { + assertResult(Array(Row(1))) { + sql("select (select 1 as b) as b").collect() + } + + assertResult(Array(Row(1))) { + sql("with t2 as (select 1 as b, 2 as c) " + + "select a from (select 1 as a union all select 2 as a) t " + + "where a = (select max(b) from t2) ").collect() + } + } + + test("correlated scalar subquery") { assertResult(Array(Row(1))) { sql("with t2 as (select 1 as b, 2 as c) " + "select a from (select 1 as a union all select 2 as a) t " + From 2d88bb30860b965af0fc3224c293808ef4580a8e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 12 Jan 2016 00:43:43 -0800 Subject: [PATCH 07/10] improve corner cases --- .../sql/catalyst/analysis/Analyzer.scala | 265 ++++++++++++------ 1 file changed, 181 insertions(+), 84 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8e25cb226b699..90d7f5317dc22 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -20,13 +20,13 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftOuter, LeftSemi} +import org.apache.spark.sql.catalyst.plans.{Inner, LeftAnti, LeftOuter, LeftSemi} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef -import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} import org.apache.spark.sql.types._ /** @@ -598,42 +598,112 @@ class Analyzer( } } + /** + * This rule resolve subqueries inside expressions, rewrite correlated subqueries or + * uncorrelated list-subquery (returns multiple rows, used with EXISTS/IN) into joins. + * + * It works as following: + * 1. For each logical plan, find out the subqueries from expressions, try to resolve them, + * update the SubQueryExpression with resolved logical plan. + * 2. For Filter, the condition will be splitted by AND, each sub-condition that has subqueries + * will be rewritten as following: + * a. EXISTS/NOT EXISTS will be rewritten as semi/anti join, unresolved condition in Filter + * will be pulled out as join conditions. + * b. IN will be rewritten as semi join, unresolved condition in Filter will be pulled out as + * join conditions, value = selected column will also be used as join condition. + * c. NOT IN will be rewritten as anti join, unresolved condition in Filter will be pulled out + * as join conditions, value = selected column will also be used as join condition, if the + * selected column is not nullable, otherwise it's not supported (raise Exception). + * d. Unresolved scalar subquery will be rewritten as left outer join, the unresolved conditoin + * in Filter will be pulled out as join condition. + */ object ResolveSubquery extends Rule[LogicalPlan] with PredicateHelper { - def hasSubquery(e: Expression): Boolean = { + private def hasSubquery(e: Expression): Boolean = { e.find(_.isInstanceOf[SubQueryExpression]).isDefined } - def onlyHasScalarSubquery(e: Expression): Boolean = { + private def onlyHasScalarSubquery(e: Expression): Boolean = { (e.find(x => x.isInstanceOf[Exists] || x.isInstanceOf[ListSubQuery]).isEmpty && e.find(_.isInstanceOf[ScalarSubQuery]).isDefined) } - def hasSubquery(q: LogicalPlan): Boolean = { + private def hasSubquery(q: LogicalPlan): Boolean = { q.expressions.exists(hasSubquery) } - def removeUnresolvedFilter(q: LogicalPlan): (LogicalPlan, Option[Expression]) = { + /** + * Removes the conjunctive predicates of Filter that can't be resolved in this logical plan, + * returns the resolved new logical plan and removed predicates. + */ + private def removeUnresolvedPredicates(q: LogicalPlan): (LogicalPlan, Option[Expression]) = { val unresolvedConditions = ArrayBuffer[Expression]() - val removed = q transformUp { - case f @ Filter(cond, child) => - val (resolved, unresolved) = splitConjunctivePredicates(cond).partition(_.resolved) - unresolvedConditions ++= unresolved - if (resolved.nonEmpty) { - Filter(resolved.reduceLeft(And), child) + var hasOuterJoin = false + val removed = q transform { + case j: Join if j.joinType != Inner => + hasOuterJoin = true + j + case f @ Filter(cond, child) if child.resolved && !f.resolved => + if (hasOuterJoin) { + // The predicates inside a outer/semi join can't be pulled over join safely. + throw new AnalysisException("accessing columns of outer query inside join is not" + + " supported") } else { - child + val (resolved, unresolved) = splitConjunctivePredicates(cond).partition(_.resolved) + unresolvedConditions ++= unresolved + if (resolved.nonEmpty) { + Filter(resolved.reduceLeft(And), child) + } else { + child + } } } - (removed, unresolvedConditions.reduceLeftOption(And)) + if (unresolvedConditions.nonEmpty) { + // try to resolve new logical plan and remove unresolved predicated again + val (removedAgain, moreConditions) = removeUnresolvedPredicates(execute(removed)) + (removedAgain, (unresolvedConditions ++ moreConditions.toSeq).reduceLeftOption(And)) + } else { + if (!q.resolved) { + throw new AnalysisException(s"subquery can't be resolved: ${q.treeString}") + } + (q, None) + } } - def rewriteInSubquery(value: Expression, subquery: LogicalPlan): (LogicalPlan, Expression) = { - val (removed, joinCondition) = removeUnresolvedFilter(execute(subquery)) - val resolved = execute(removed) - assert(resolved.resolved) - val right = resolved.output.head - val equalCond = EqualTo(value, right) + /** + * Returns a resolved subquery and predicate that will be used to rewrite the IN subquery as + * semi join (predicate will be used as join condition). + */ + private def rewriteInSubquery( + value: Expression, + subquery: LogicalPlan): (LogicalPlan, Expression) = { + val (resolved, joinCondition) = removeUnresolvedPredicates(execute(subquery)) + // check the dataType of value and subquery + val equalCond = value match { + case CreateStruct(columns) => + if (columns.length != resolved.output.length) { + throw new AnalysisException(s"the number of columns in value (${columns.length}) does" + + s" not match with the number of columns in subquery (${resolved.output.length})") + } + columns.zip(resolved.output).map { + case (e, attr) => + if (e.dataType != attr.dataType) { + throw new AnalysisException(s"data type of value (${e.dataType}) does not match" + + s" with subquery (${attr.dataType})") + } + EqualTo(e, attr) + }.reduceLeft(And) + case e => + if (resolved.output.length != 1) { + throw new AnalysisException(s"the number of columns in value (1) does" + + s" not match with the number of columns in subquery (${resolved.output.length})") + } + if (e.dataType != resolved.output.head.dataType) { + throw new AnalysisException(s"data type of value (${e.dataType}) does not match" + + s" with subquery (${resolved.output.head.dataType})") + } + EqualTo(value, resolved.output.head) + } val cond = if (joinCondition.isDefined){ And(joinCondition.get, equalCond) } else { @@ -650,70 +720,97 @@ class Analyzer( e.withNewPlan(execute(e.query)) } - if (afterResolve.resolved) { - afterResolve - } else { - afterResolve match { - case f @ Filter(condition, child) => - val (withSubquery, withoutSubquery: Seq[Expression]) = - splitConjunctivePredicates(condition).partition(hasSubquery) - val newConds = ArrayBuffer[Expression]() - var newChild: LogicalPlan = child - withSubquery.foreach { - case Exists(sub) => - // use all the conditions as join condition - val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) - newChild = Join(newChild, removed, LeftSemi, joinCondition) - case Not(Exists(sub)) => - val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) - newChild = Join(newChild, removed, LeftAnti, joinCondition) - case In(value, ListSubQuery(sub) :: Nil) => - val (resolved, cond) = rewriteInSubquery(value, sub) - newChild = Join(newChild, resolved, LeftSemi, Some(cond)) - case Not(In(value, ListSubQuery(sub) :: Nil)) => - val (resolved, cond) = rewriteInSubquery(value, sub) - newChild = Join(newChild, resolved, LeftAnti, Some(cond)) - case expr: Expression if onlyHasScalarSubquery(expr) => - val newCond = expr.transformUp { - case ScalarSubQuery(sub) if !sub.resolved => - val (removed, joinCondition) = removeUnresolvedFilter(execute(sub)) - val (internalExprs, externalJoinExprs) = - splitConjunctivePredicates(joinCondition.get). - flatMap(_.children).partition(_.resolved) - val joinAttrs: Seq[NamedExpression] = internalExprs.map { - case e: NamedExpression => e - case e => Alias(e, e.toString)() - } - val newCond = joinAttrs.zip(externalJoinExprs).map { - case (attr, expr) => EqualTo(expr, attr.toAttribute) - }.reduceLeftOption(And) - val resolved = execute(removed) - assert(resolved.resolved) - val newAgg = resolved match { - // the scalar subquery can have not grouping keys and single - // AggregateExpression - case Aggregate(Nil, aggregateExpression :: Nil, child) => - Aggregate( - internalExprs, - joinAttrs ++ Seq(aggregateExpression), - child) - } - newChild = Join(newChild, newAgg, LeftOuter, newCond) - val output = resolved.output.head - output - } - newConds += newCond - case other => - // TODO: support Or(Exists(), Exists()) - sys.error(s"not supported subquery: $other") - } - if (withoutSubquery.nonEmpty || newConds.nonEmpty) { - Filter((withoutSubquery ++ newConds).reduceLeft(And), newChild) - } else { - newChild - } - case other => other - } + afterResolve match { + case f @ Filter(condition, child) => + + val (withSubquery, withoutSubquery: Seq[Expression]) = + splitConjunctivePredicates(condition).partition(hasSubquery) + val newConds = ArrayBuffer[Expression]() + var newChild: LogicalPlan = child + + withSubquery.foreach { + case Exists(sub) => + // use all the predicates as join condition + val (resolved, joinCondition) = removeUnresolvedPredicates(sub) + newChild = Join(newChild, resolved, LeftSemi, joinCondition) + case Not(Exists(sub)) => + val (resolved, joinCondition) = removeUnresolvedPredicates(sub) + newChild = Join(newChild, resolved, LeftAnti, joinCondition) + + case In(value, ListSubQuery(sub) :: Nil) => + val (resolved, cond) = rewriteInSubquery(value, sub) + newChild = Join(newChild, resolved, LeftSemi, Some(cond)) + + case Not(In(value, ListSubQuery(sub) :: Nil)) => + val (resolved, cond) = rewriteInSubquery(value, sub) + if (resolved.output.exists(_.nullable)) { + throw new AnalysisException(s"NOT IN with nullable subquery is not supported") + } + if (value.nullable) { + // Currently, left anti join will output left rows if value is null, we should + // filter that out before left anti join (because NOT EXISTS requires that) + newChild = Filter(Not(IsNull(value)), newChild) + } + newChild = Join(newChild, resolved, LeftAnti, Some(cond)) + + case expr: Expression if onlyHasScalarSubquery(expr) => + // rewrites the correlated scalar subqueries as join + val newCond = expr.transformUp { + case ScalarSubQuery(sub) if !sub.resolved => + + val (resolved, joinCondition) = removeUnresolvedPredicates(sub) + val predicates = splitConjunctivePredicates(joinCondition.get) + if (predicates.exists(e => !e.isInstanceOf[EqualTo])) { + throw new AnalysisException(s"only external column used in equal are" + + s" supported") + } + val (internalExprs, externalJoinExprs) = + predicates.flatMap(_.children).partition(_.resolved) + + // Use the expression from subquery as grouping key + val groupingExpr: Seq[NamedExpression] = internalExprs.map { + case e: NamedExpression => e + case e => Alias(e, e.toString)() + } + // rewrite the join condition to use attribute + val newCondition = externalJoinExprs.zip(groupingExpr).map { + case (expr, attr) => EqualTo(expr, attr.toAttribute) + }.reduceLeftOption(And) + val newAgg = resolved match { + // the scalar subquery can have not grouping keys and single + // AggregateExpression + case Aggregate(Nil, aggregateExpression :: Nil, child) => + Aggregate( + groupingExpr, + groupingExpr ++ Seq(aggregateExpression), + child) + } + newChild = Join(newChild, newAgg, LeftOuter, newCondition) + // rewrite current scalar subquery as the single output from original + // aggregation, which became the last of output of newChild. + newChild.output.last + } + newConds += newCond + + case other => + throw new AnalysisException(s"EXISTS/IN only be used as top level predicate " + + s"(with AND)") + } + + if (withoutSubquery.nonEmpty || newConds.nonEmpty) { + Filter((withoutSubquery ++ newConds).reduceLeft(And), newChild) + } else { + newChild + } + + case other => + if (other.expressions.exists(_.find(_.isInstanceOf[Exists]).isDefined)) { + throw new AnalysisException(s"EXISTS subquery can't be used inside $other") + } + if (other.expressions.exists(_.find(_.isInstanceOf[ListSubQuery]).isDefined)) { + throw new AnalysisException(s"IN subquery can't be used inside $other") + } + other } } } From 6809ad5963f4d4ae63736033c306cb7f6b2303a2 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 12 Jan 2016 10:08:04 -0800 Subject: [PATCH 08/10] update style --- .../src/main/scala/org/apache/spark/sql/DataFrame.scala | 6 +++--- .../spark/sql/execution/joins/BroadcastSemiJoin.scala | 2 +- .../apache/spark/sql/execution/joins/SemiJoinSuite.scala | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 918192debd5dc..f056b51df2713 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -29,15 +29,15 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.PythonRDD import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection, SqlParser} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection, SqlParser} -import org.apache.spark.sql.execution.datasources.json.JacksonGenerator -import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation} import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, FileRelation, LogicalRDD, QueryExecution, Queryable, SQLExecution} +import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation} +import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala index 77d12971edae3..c6dcfe2547199 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.{InternalAccumulator, TaskContext} +import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala index d734457a89c0c..2d032f78b9a46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.sql.{DataFrame, Row, SQLConf} import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys -import org.apache.spark.sql.catalyst.plans.logical.Join import org.apache.spark.sql.catalyst.plans.{JoinType, LeftAnti, LeftSemi} +import org.apache.spark.sql.catalyst.plans.logical.Join import org.apache.spark.sql.execution.{EnsureRequirements, SparkPlan, SparkPlanTest} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} -import org.apache.spark.sql.{DataFrame, Row, SQLConf} class SemiJoinSuite extends SparkPlanTest with SharedSQLContext { From d1feebd276fe620fb162ad94ed3d544b6b075366 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 13 Jan 2016 00:31:07 -0800 Subject: [PATCH 09/10] add more tests --- .../sql/catalyst/analysis/Analyzer.scala | 16 +++++---- .../spark/sql/catalyst/CatalystQlSuite.scala | 34 +++++++++---------- .../spark/sql/execution/SparkStrategies.scala | 2 +- ....scala => BroadcastLeftSemiJoinHash.scala} | 2 +- .../org/apache/spark/sql/JoinSuite.scala | 4 +-- .../execution/joins/BroadcastJoinSuite.scala | 2 +- .../sql/execution/joins/SemiJoinSuite.scala | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 4 +-- .../sql/hive/execution/HiveQuerySuite.scala | 20 ++++++++++- 9 files changed, 54 insertions(+), 32 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/joins/{BroadcastSemiJoin.scala => BroadcastLeftSemiJoinHash.scala} (98%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9825325caddbb..7eb5789307ecd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -75,10 +75,10 @@ class Analyzer( ResolvePivot :: ResolveUpCast :: ResolveSortReferences :: - ResolveSubquery :: ResolveGenerate :: ResolveFunctions :: ResolveAliases :: + ResolveSubquery :: ResolveWindowOrder :: ResolveWindowFrame :: ExtractWindowExpressions :: @@ -682,7 +682,7 @@ class Analyzer( val equalCond = value match { case CreateStruct(columns) => if (columns.length != resolved.output.length) { - throw new AnalysisException(s"the number of columns in value (${columns.length}) does" + + throw new AnalysisException(s"the number of fields in value (${columns.length}) does" + s" not match with the number of columns in subquery (${resolved.output.length})") } columns.zip(resolved.output).map { @@ -737,11 +737,11 @@ class Analyzer( val (resolved, joinCondition) = removeUnresolvedPredicates(sub) newChild = Join(newChild, resolved, LeftAnti, joinCondition) - case In(value, ListSubQuery(sub) :: Nil) => + case In(value, ListSubQuery(sub) :: Nil) if value.resolved => val (resolved, cond) = rewriteInSubquery(value, sub) newChild = Join(newChild, resolved, LeftSemi, Some(cond)) - case Not(In(value, ListSubQuery(sub) :: Nil)) => + case Not(In(value, ListSubQuery(sub) :: Nil)) if value.resolved => val (resolved, cond) = rewriteInSubquery(value, sub) if (resolved.output.exists(_.nullable)) { throw new AnalysisException(s"NOT IN with nullable subquery is not supported") @@ -793,8 +793,12 @@ class Analyzer( newConds += newCond case other => - throw new AnalysisException(s"EXISTS/IN only be used as top level predicate " + - s"(with AND)") + if (other.find(_.isInstanceOf[Exists]).isDefined) { + throw new AnalysisException(s"EXISTS only be used as top level predicate " + + s"(with AND)") + } + // others could be resolved later + newConds += other } if (withoutSubquery.nonEmpty || newConds.nonEmpty) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala index 60aacb88ff907..0b99079eeb7f9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala @@ -172,22 +172,22 @@ class CatalystQlSuite extends PlanTest { } test("subquery") { - parser.createPlan("select (select max(b) from s) ss from t") - - parser.createPlan("select * from t where a = (select b from s)") - parser.createPlan("select * from t where a > (select b from s)") - parser.createPlan("select * from t where a in (select b from s)") - parser.createPlan("select * from t where a not in (select b from s)") - parser.createPlan("select * from t where a and exists (select b from s)") - parser.createPlan("select * from t where not exists (select b from s)") - parser.createPlan("select * from t where exists (select b from s)") - - parser.createPlan("select * from t group by g having a = (select b from s)") - parser.createPlan("select * from t group by g having a > (select b from s)") - parser.createPlan("select * from t group by g having a in (select b from s)") - parser.createPlan("select * from t group by g having a not in (select b from s)") - parser.createPlan("select * from t group by g having exists (select b from s)") - parser.createPlan("select * from t group by g having not exists (select b from s)") - parser.createPlan("select * from t group by g having exists (select b from s)") + parser.parsePlan("select (select max(b) from s) ss from t") + + parser.parsePlan("select * from t where a = (select b from s)") + parser.parsePlan("select * from t where a > (select b from s)") + parser.parsePlan("select * from t where a in (select b from s)") + parser.parsePlan("select * from t where a not in (select b from s)") + parser.parsePlan("select * from t where a and exists (select b from s)") + parser.parsePlan("select * from t where not exists (select b from s)") + parser.parsePlan("select * from t where exists (select b from s)") + + parser.parsePlan("select * from t group by g having a = (select b from s)") + parser.parsePlan("select * from t group by g having a > (select b from s)") + parser.parsePlan("select * from t group by g having a in (select b from s)") + parser.parsePlan("select * from t group by g having a not in (select b from s)") + parser.parsePlan("select * from t group by g having exists (select b from s)") + parser.parsePlan("select * from t group by g having not exists (select b from s)") + parser.parsePlan("select * from t group by g having exists (select b from s)") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 4fdab3e7c7cd4..d3e89c1290e8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -36,7 +36,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case ExtractEquiJoinKeys(joinType @ (LeftSemi | LeftAnti), leftKeys, rightKeys, condition, left, CanBroadcast(right)) => - joins.BroadcastSemiJoin( + joins.BroadcastLeftSemiJoinHash( joinType, leftKeys, rightKeys, planLater(left), planLater(right), condition) :: Nil // Find left semi joins where at least some predicates can be evaluated by matching join keys case ExtractEquiJoinKeys( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index 54d21603bf29c..2af3b47ae7e6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastSemiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics * Build the right table's join keys into a HashSet, and iteratively go through the left * table, to find the if join keys are in the Hash set. */ -case class BroadcastSemiJoin( +case class BroadcastLeftSemiJoinHash( joinType: JoinType, leftKeys: Seq[Expression], rightKeys: Seq[Expression], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 6233759723a9b..9a3c262e9485d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -50,7 +50,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { case j: LeftSemiJoinBNL => j case j: CartesianProduct => j case j: BroadcastNestedLoopJoin => j - case j: BroadcastSemiJoin => j + case j: BroadcastLeftSemiJoinHash => j case j: SortMergeJoin => j case j: SortMergeOuterJoin => j } @@ -423,7 +423,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000000000") { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", - classOf[BroadcastSemiJoin]) + classOf[BroadcastLeftSemiJoinHash]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index 26328a4f52473..58581d71e1bc1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -77,7 +77,7 @@ class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { } test("unsafe broadcast left semi join updates peak execution memory") { - testBroadcastJoin[BroadcastSemiJoin]("unsafe broadcast left semi join", "leftsemi") + testBroadcastJoin[BroadcastLeftSemiJoinHash]("unsafe broadcast left semi join", "leftsemi") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala index 2d032f78b9a46..a0ceb0b4cbb07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala @@ -87,7 +87,7 @@ class SemiJoinSuite extends SparkPlanTest with SharedSQLContext { extractJoinParts().foreach { case (joinType, leftKeys, rightKeys, boundCondition, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - BroadcastSemiJoin(joinType, leftKeys, rightKeys, left, right, boundCondition), + BroadcastLeftSemiJoinHash(joinType, leftKeys, rightKeys, left, right, boundCondition), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 23d58801feafc..78f74cdc19ddb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -207,7 +207,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. var bhj = df.queryExecution.sparkPlan.collect { - case j: BroadcastSemiJoin => j + case j: BroadcastLeftSemiJoinHash => j } assert(bhj.size === 1, s"actual query plans do not contain broadcast join: ${df.queryExecution}") @@ -220,7 +220,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1") df = sql(leftSemiJoinQuery) bhj = df.queryExecution.sparkPlan.collect { - case j: BroadcastSemiJoin => j + case j: BroadcastLeftSemiJoinHash => j } assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index fa60225125fcb..d601eb689d817 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -815,6 +815,11 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("select * from (select 1 as a union all select 2 as a) t " + "where not exists (select * from (select 1 as b) t2 where b = a and b < 2) ").collect() } + intercept[AnalysisException] { + sql("select * from (select 1 as a union all select 2 as a) t " + + "where exists (select * from (select 1 as b) t2 where b = a and b < 2)" + + " or not exists (select 1) ").collect() + } } test("in subquery") { @@ -823,8 +828,21 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { "select * from (select 1 as a union all select 2 as a) t " + "where a in (select b as a from t2 where b < 2) ").collect() } - assertResult(Array(Row(2))) { + assertResult(Array(Row(1, 2))) { sql("with t2 as (select 1 as b) " + + "select * from (select 1 as a, 2 as b) t " + + "where struct(a, b) in (select b, b + 1 from t2 where b < 2) ").collect() + } + } + + test("not in subquery") { + assertResult(Array(Row(1))) { + sql("with t2 as (select 1 as b) " + + "select * from (select 1 as a union all select null as a) t " + + "where a not in (select b + 1 from t2 where b < 2) ").collect() + } + intercept[AnalysisException] { + sql("with t2 as (select 1 as b unoin all select null as b) " + "select * from (select 1 as a union all select 2 as a) t " + "where a not in (select b as a from t2 where b < 2) ").collect() } From 56d140f471b782f7bbcbf08104a25163f6c3c2a1 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 13 Jan 2016 01:24:20 -0800 Subject: [PATCH 10/10] add Hive tests --- .../execution/HiveCompatibilitySuite.scala | 7 + ...ISTS #1-0-8cad7cc250640b9ed90f55936eaf7f6b | 5 + ...ISTS #2-0-1d4cf28a3e6b326d3d2d86d36aa2155e | 5 + ...h IN #1-0-cb486c695b860ddae0a8887f04576a95 | 5 + ...h IN #2-0-144533cde4a3663bc76a49cff5b2f4ed | 5 + ...ISTS #1-0-c3899f6cc53b07b33290bce25141797f | 5 + ...ISTS #2-0-25011e3c7dfa7ef5bbea06693af3ab13 | 0 ...ISTS #2-0-88c9747ba816791fd56b5feb433678cc | 0 ...T IN #1-0-5057c72686a27eb5db4ae61986158c9e | 5 + ...T IN #2-0-36dabad40171b9ecacc98d97ad0347bc | 0 ...h IN #1-0-2803faacd6946617b6384712e6352554 | 5 + ...h IN #2-0-c061a63f67236e9475e3d2ed84ef939b | 5 + ...h IN #3-0-b1025666e1db27739b422775ffb1347b | 5 + ...h IN #4-0-4c1fdc5fdd9c1abc347ccfa22e84b43b | 5 + ...h IN #5-0-4daf3a47b26cdb1984726cd21c492c1f | 5 + ...h IN #6-0-f2bfe78e87e8300e4a662cd0feea34cd | 5 + ...h IN #7-0-3a5282e8d423c938a7b23afbc42a04e7 | 0 ...T IN #1-0-58e3b9feb76cf7ca99c62e83fe819fbe | 0 ...T IN #2-0-1b812a2ca72a3ce30622193d3d880667 | 5 + ...OT IN #2-0-80d53a89ed6c0fd97b3529a1e8911c4 | 0 ...T IN #3-0-c6697fafe394f6fb0cb22c3cdf374d1b | 5 + ...T IN #4-0-68a0bf11d44d02bb0c0c4d7b4b689e5c | 0 ...T IN #4-0-e50524ec97b9fde60cead688af4bfe32 | 5 + ...T IN #5-0-e65a783d31092fdbd4331a0ca58cf20c | 4 + ...T IN #6-0-6f20727475d71acac93607ccc01ab743 | 0 ...T IN #6-0-9b5cecdc5a285f5669abcee52955b3ba | 5 + ...T IN #7-0-a1ea59ae43e4392cfdb40917d7a25204 | 5 + ..._exists-0-71049df380c600f02fb6c00d19999e8d | 0 ...y_exists-1-57688cd1babd6a79bc3b2d2ec434b39 | 11 + ..._exists-2-4c686f9b9cf51ae1b369acfa43d6c73f | 0 ..._exists-3-da5828589960a60826f5a08948850d78 | 11 + ..._exists-4-2058d464561ef7b24d896ec8ecb21a00 | 11 + ..._having-0-927435f429722c2de003e376b9f0bbd2 | 0 ..._having-1-b7ac11dbf892c229e180a2bc761117fe | 6 + ..._having-2-4f0b2dbae1324cdc5f3ead83b632e503 | 0 ..._having-3-da5828589960a60826f5a08948850d78 | 6 + ..._having-4-fd5457ec549cc2265848f3c95a60693d | 6 + ..._having-5-aafe13388d5795b26035167edd90a69b | 6 + ..._having-0-dda16565b98926fc3587de937b9401c7 | 0 ...n_having-1-374e39786feb745cd70f25be58bfa24 | 0 ...having-10-b8ded52f10f8103684cda7bba20d2201 | 0 ...having-11-ddeeedb49ded9eb733a4792fff83abe4 | 0 ..._having-2-877cbfc817ff3718f65073378a0c0829 | 0 ..._having-3-63a96439d273b9ad3304d3036bd79e35 | 303 ++++++++++++++ ..._having-4-5d1259d48aa4b26931f1dbe686a0d2d7 | 31 ++ ..._having-5-1beb605f3b9b0825c69dc5f52d085225 | 0 ..._having-6-9543704852a4d71a85b90b85a0c5c0a5 | 0 ..._having-7-6bba00f0273f13733fadbe10b43876f5 | 6 + ..._having-8-662f1f7435da5d66fd4b09244387c06b | 0 ..._having-9-24ca942f094b14b92086305cc125e833 | 1 + ...texists-0-75cd3855b33f05667ae76896f4b25d3d | 0 ...texists-1-4ae5bcc868eb27add076db2cb3ca9678 | 119 ++++++ ...texists-2-73a67f6cae6d8e68efebdab4fbade162 | 0 ...texists-3-a8b49a691e12360c7c3fa5df113ba8cf | 119 ++++++ ..._having-0-872612e3ae6ef1445982517a94200075 | 0 ..._having-1-8f6c09c8a89cc5939c1c309d660e7b3e | 14 + ..._having-2-fb172ff54d6814f42360cb9f30f4882e | 0 ...s_having-3-edd8e7bbc4bfde58cf744fc0901e2ac | 14 + ..._having-0-d3f50875bd5dff172cf813fdb7d738eb | 0 ..._having-1-dda16565b98926fc3587de937b9401c7 | 0 ...n_having-2-374e39786feb745cd70f25be58bfa24 | 0 ..._having-3-21a44539fd357dc260687003554fe02a | 0 ..._having-4-dea2fabba75cc13e7fa8df072f6b557b | 0 .../hive/execution/HiveSubquerySuite.scala | 391 ++++++++++++++++++ 64 files changed, 1151 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/(correlated)WHERE clause with EXISTS #1-0-8cad7cc250640b9ed90f55936eaf7f6b create mode 100644 sql/hive/src/test/resources/golden/(correlated)WHERE clause with EXISTS #2-0-1d4cf28a3e6b326d3d2d86d36aa2155e create mode 100644 sql/hive/src/test/resources/golden/(correlated)WHERE clause with IN #1-0-cb486c695b860ddae0a8887f04576a95 create mode 100644 sql/hive/src/test/resources/golden/(correlated)WHERE clause with IN #2-0-144533cde4a3663bc76a49cff5b2f4ed create mode 100644 sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #1-0-c3899f6cc53b07b33290bce25141797f create mode 100644 sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #2-0-25011e3c7dfa7ef5bbea06693af3ab13 create mode 100644 sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #2-0-88c9747ba816791fd56b5feb433678cc create mode 100644 sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT IN #1-0-5057c72686a27eb5db4ae61986158c9e create mode 100644 sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT IN #2-0-36dabad40171b9ecacc98d97ad0347bc create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #1-0-2803faacd6946617b6384712e6352554 create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #2-0-c061a63f67236e9475e3d2ed84ef939b create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #3-0-b1025666e1db27739b422775ffb1347b create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #4-0-4c1fdc5fdd9c1abc347ccfa22e84b43b create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #5-0-4daf3a47b26cdb1984726cd21c492c1f create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #6-0-f2bfe78e87e8300e4a662cd0feea34cd create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #7-0-3a5282e8d423c938a7b23afbc42a04e7 create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #1-0-58e3b9feb76cf7ca99c62e83fe819fbe create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #2-0-1b812a2ca72a3ce30622193d3d880667 create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #2-0-80d53a89ed6c0fd97b3529a1e8911c4 create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #3-0-c6697fafe394f6fb0cb22c3cdf374d1b create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #4-0-68a0bf11d44d02bb0c0c4d7b4b689e5c create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #4-0-e50524ec97b9fde60cead688af4bfe32 create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #5-0-e65a783d31092fdbd4331a0ca58cf20c create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #6-0-6f20727475d71acac93607ccc01ab743 create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #6-0-9b5cecdc5a285f5669abcee52955b3ba create mode 100644 sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #7-0-a1ea59ae43e4392cfdb40917d7a25204 create mode 100644 sql/hive/src/test/resources/golden/subquery_exists-0-71049df380c600f02fb6c00d19999e8d create mode 100644 sql/hive/src/test/resources/golden/subquery_exists-1-57688cd1babd6a79bc3b2d2ec434b39 create mode 100644 sql/hive/src/test/resources/golden/subquery_exists-2-4c686f9b9cf51ae1b369acfa43d6c73f create mode 100644 sql/hive/src/test/resources/golden/subquery_exists-3-da5828589960a60826f5a08948850d78 create mode 100644 sql/hive/src/test/resources/golden/subquery_exists-4-2058d464561ef7b24d896ec8ecb21a00 create mode 100644 sql/hive/src/test/resources/golden/subquery_exists_having-0-927435f429722c2de003e376b9f0bbd2 create mode 100644 sql/hive/src/test/resources/golden/subquery_exists_having-1-b7ac11dbf892c229e180a2bc761117fe create mode 100644 sql/hive/src/test/resources/golden/subquery_exists_having-2-4f0b2dbae1324cdc5f3ead83b632e503 create mode 100644 sql/hive/src/test/resources/golden/subquery_exists_having-3-da5828589960a60826f5a08948850d78 create mode 100644 sql/hive/src/test/resources/golden/subquery_exists_having-4-fd5457ec549cc2265848f3c95a60693d create mode 100644 sql/hive/src/test/resources/golden/subquery_exists_having-5-aafe13388d5795b26035167edd90a69b create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-0-dda16565b98926fc3587de937b9401c7 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-1-374e39786feb745cd70f25be58bfa24 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-10-b8ded52f10f8103684cda7bba20d2201 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-11-ddeeedb49ded9eb733a4792fff83abe4 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-2-877cbfc817ff3718f65073378a0c0829 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-3-63a96439d273b9ad3304d3036bd79e35 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-4-5d1259d48aa4b26931f1dbe686a0d2d7 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-5-1beb605f3b9b0825c69dc5f52d085225 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-6-9543704852a4d71a85b90b85a0c5c0a5 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-7-6bba00f0273f13733fadbe10b43876f5 create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-8-662f1f7435da5d66fd4b09244387c06b create mode 100644 sql/hive/src/test/resources/golden/subquery_in_having-9-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/subquery_notexists-0-75cd3855b33f05667ae76896f4b25d3d create mode 100644 sql/hive/src/test/resources/golden/subquery_notexists-1-4ae5bcc868eb27add076db2cb3ca9678 create mode 100644 sql/hive/src/test/resources/golden/subquery_notexists-2-73a67f6cae6d8e68efebdab4fbade162 create mode 100644 sql/hive/src/test/resources/golden/subquery_notexists-3-a8b49a691e12360c7c3fa5df113ba8cf create mode 100644 sql/hive/src/test/resources/golden/subquery_notexists_having-0-872612e3ae6ef1445982517a94200075 create mode 100644 sql/hive/src/test/resources/golden/subquery_notexists_having-1-8f6c09c8a89cc5939c1c309d660e7b3e create mode 100644 sql/hive/src/test/resources/golden/subquery_notexists_having-2-fb172ff54d6814f42360cb9f30f4882e create mode 100644 sql/hive/src/test/resources/golden/subquery_notexists_having-3-edd8e7bbc4bfde58cf744fc0901e2ac create mode 100644 sql/hive/src/test/resources/golden/subquery_notin_having-0-d3f50875bd5dff172cf813fdb7d738eb create mode 100644 sql/hive/src/test/resources/golden/subquery_notin_having-1-dda16565b98926fc3587de937b9401c7 create mode 100644 sql/hive/src/test/resources/golden/subquery_notin_having-2-374e39786feb745cd70f25be58bfa24 create mode 100644 sql/hive/src/test/resources/golden/subquery_notin_having-3-21a44539fd357dc260687003554fe02a create mode 100644 sql/hive/src/test/resources/golden/subquery_notin_having-4-dea2fabba75cc13e7fa8df072f6b557b create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSubquerySuite.scala diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index afd2f611580fc..ef731b4f4ae5e 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -859,6 +859,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "stats_empty_partition", "stats_publisher_error_1", "subq2", + "subquery_exists", + "subquery_exists_having", + "subquery_notexists", + "subquery_notexists_having", + "subquery_in", + "subquery_in_having", + "subquery_notin_having", "tablename_with_select", "timestamp_3", "timestamp_comparison", diff --git a/sql/hive/src/test/resources/golden/(correlated)WHERE clause with EXISTS #1-0-8cad7cc250640b9ed90f55936eaf7f6b b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with EXISTS #1-0-8cad7cc250640b9ed90f55936eaf7f6b new file mode 100644 index 0000000000000..1851dbf86fb6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with EXISTS #1-0-8cad7cc250640b9ed90f55936eaf7f6b @@ -0,0 +1,5 @@ +10 val_10 +11 val_11 +12 val_12 +12 val_12 +15 val_15 diff --git a/sql/hive/src/test/resources/golden/(correlated)WHERE clause with EXISTS #2-0-1d4cf28a3e6b326d3d2d86d36aa2155e b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with EXISTS #2-0-1d4cf28a3e6b326d3d2d86d36aa2155e new file mode 100644 index 0000000000000..322853999e0da --- /dev/null +++ b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with EXISTS #2-0-1d4cf28a3e6b326d3d2d86d36aa2155e @@ -0,0 +1,5 @@ +17 val_17 +18 val_18 +18 val_18 +19 val_19 +20 val_20 diff --git a/sql/hive/src/test/resources/golden/(correlated)WHERE clause with IN #1-0-cb486c695b860ddae0a8887f04576a95 b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with IN #1-0-cb486c695b860ddae0a8887f04576a95 new file mode 100644 index 0000000000000..1851dbf86fb6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with IN #1-0-cb486c695b860ddae0a8887f04576a95 @@ -0,0 +1,5 @@ +10 val_10 +11 val_11 +12 val_12 +12 val_12 +15 val_15 diff --git a/sql/hive/src/test/resources/golden/(correlated)WHERE clause with IN #2-0-144533cde4a3663bc76a49cff5b2f4ed b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with IN #2-0-144533cde4a3663bc76a49cff5b2f4ed new file mode 100644 index 0000000000000..322853999e0da --- /dev/null +++ b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with IN #2-0-144533cde4a3663bc76a49cff5b2f4ed @@ -0,0 +1,5 @@ +17 val_17 +18 val_18 +18 val_18 +19 val_19 +20 val_20 diff --git a/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #1-0-c3899f6cc53b07b33290bce25141797f b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #1-0-c3899f6cc53b07b33290bce25141797f new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #1-0-c3899f6cc53b07b33290bce25141797f @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #2-0-25011e3c7dfa7ef5bbea06693af3ab13 b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #2-0-25011e3c7dfa7ef5bbea06693af3ab13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #2-0-88c9747ba816791fd56b5feb433678cc b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT EXISTS #2-0-88c9747ba816791fd56b5feb433678cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT IN #1-0-5057c72686a27eb5db4ae61986158c9e b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT IN #1-0-5057c72686a27eb5db4ae61986158c9e new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT IN #1-0-5057c72686a27eb5db4ae61986158c9e @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT IN #2-0-36dabad40171b9ecacc98d97ad0347bc b/sql/hive/src/test/resources/golden/(correlated)WHERE clause with NOT IN #2-0-36dabad40171b9ecacc98d97ad0347bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #1-0-2803faacd6946617b6384712e6352554 b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #1-0-2803faacd6946617b6384712e6352554 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #1-0-2803faacd6946617b6384712e6352554 @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #2-0-c061a63f67236e9475e3d2ed84ef939b b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #2-0-c061a63f67236e9475e3d2ed84ef939b new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #2-0-c061a63f67236e9475e3d2ed84ef939b @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #3-0-b1025666e1db27739b422775ffb1347b b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #3-0-b1025666e1db27739b422775ffb1347b new file mode 100644 index 0000000000000..1851dbf86fb6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #3-0-b1025666e1db27739b422775ffb1347b @@ -0,0 +1,5 @@ +10 val_10 +11 val_11 +12 val_12 +12 val_12 +15 val_15 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #4-0-4c1fdc5fdd9c1abc347ccfa22e84b43b b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #4-0-4c1fdc5fdd9c1abc347ccfa22e84b43b new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #4-0-4c1fdc5fdd9c1abc347ccfa22e84b43b @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #5-0-4daf3a47b26cdb1984726cd21c492c1f b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #5-0-4daf3a47b26cdb1984726cd21c492c1f new file mode 100644 index 0000000000000..b32ee538203af --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #5-0-4daf3a47b26cdb1984726cd21c492c1f @@ -0,0 +1,5 @@ +4 val_4 +5 val_5 +5 val_5 +5 val_5 +8 val_8 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #6-0-f2bfe78e87e8300e4a662cd0feea34cd b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #6-0-f2bfe78e87e8300e4a662cd0feea34cd new file mode 100644 index 0000000000000..9f2a1dbfaaba9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #6-0-f2bfe78e87e8300e4a662cd0feea34cd @@ -0,0 +1,5 @@ +11 val_11 +12 val_12 +12 val_12 +15 val_15 +15 val_15 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #7-0-3a5282e8d423c938a7b23afbc42a04e7 b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with IN #7-0-3a5282e8d423c938a7b23afbc42a04e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #1-0-58e3b9feb76cf7ca99c62e83fe819fbe b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #1-0-58e3b9feb76cf7ca99c62e83fe819fbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #2-0-1b812a2ca72a3ce30622193d3d880667 b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #2-0-1b812a2ca72a3ce30622193d3d880667 new file mode 100644 index 0000000000000..a4c793d0a1ab8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #2-0-1b812a2ca72a3ce30622193d3d880667 @@ -0,0 +1,5 @@ +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #2-0-80d53a89ed6c0fd97b3529a1e8911c4 b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #2-0-80d53a89ed6c0fd97b3529a1e8911c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #3-0-c6697fafe394f6fb0cb22c3cdf374d1b b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #3-0-c6697fafe394f6fb0cb22c3cdf374d1b new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #3-0-c6697fafe394f6fb0cb22c3cdf374d1b @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #4-0-68a0bf11d44d02bb0c0c4d7b4b689e5c b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #4-0-68a0bf11d44d02bb0c0c4d7b4b689e5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #4-0-e50524ec97b9fde60cead688af4bfe32 b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #4-0-e50524ec97b9fde60cead688af4bfe32 new file mode 100644 index 0000000000000..126d803db5377 --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #4-0-e50524ec97b9fde60cead688af4bfe32 @@ -0,0 +1,5 @@ +33 val_33 +34 val_34 +35 val_35 +35 val_35 +35 val_35 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #5-0-e65a783d31092fdbd4331a0ca58cf20c b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #5-0-e65a783d31092fdbd4331a0ca58cf20c new file mode 100644 index 0000000000000..f396cf48c36c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #5-0-e65a783d31092fdbd4331a0ca58cf20c @@ -0,0 +1,4 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #6-0-6f20727475d71acac93607ccc01ab743 b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #6-0-6f20727475d71acac93607ccc01ab743 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #6-0-9b5cecdc5a285f5669abcee52955b3ba b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #6-0-9b5cecdc5a285f5669abcee52955b3ba new file mode 100644 index 0000000000000..126d803db5377 --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #6-0-9b5cecdc5a285f5669abcee52955b3ba @@ -0,0 +1,5 @@ +33 val_33 +34 val_34 +35 val_35 +35 val_35 +35 val_35 diff --git a/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #7-0-a1ea59ae43e4392cfdb40917d7a25204 b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #7-0-a1ea59ae43e4392cfdb40917d7a25204 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/(unrelated)WHERE clause with NOT IN #7-0-a1ea59ae43e4392cfdb40917d7a25204 @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/sql/hive/src/test/resources/golden/subquery_exists-0-71049df380c600f02fb6c00d19999e8d b/sql/hive/src/test/resources/golden/subquery_exists-0-71049df380c600f02fb6c00d19999e8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_exists-1-57688cd1babd6a79bc3b2d2ec434b39 b/sql/hive/src/test/resources/golden/subquery_exists-1-57688cd1babd6a79bc3b2d2ec434b39 new file mode 100644 index 0000000000000..7babf117d853b --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists-1-57688cd1babd6a79bc3b2d2ec434b39 @@ -0,0 +1,11 @@ +98 val_98 +92 val_92 +96 val_96 +95 val_95 +98 val_98 +90 val_90 +95 val_95 +90 val_90 +97 val_97 +90 val_90 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/subquery_exists-2-4c686f9b9cf51ae1b369acfa43d6c73f b/sql/hive/src/test/resources/golden/subquery_exists-2-4c686f9b9cf51ae1b369acfa43d6c73f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_exists-3-da5828589960a60826f5a08948850d78 b/sql/hive/src/test/resources/golden/subquery_exists-3-da5828589960a60826f5a08948850d78 new file mode 100644 index 0000000000000..7babf117d853b --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists-3-da5828589960a60826f5a08948850d78 @@ -0,0 +1,11 @@ +98 val_98 +92 val_92 +96 val_96 +95 val_95 +98 val_98 +90 val_90 +95 val_95 +90 val_90 +97 val_97 +90 val_90 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/subquery_exists-4-2058d464561ef7b24d896ec8ecb21a00 b/sql/hive/src/test/resources/golden/subquery_exists-4-2058d464561ef7b24d896ec8ecb21a00 new file mode 100644 index 0000000000000..7babf117d853b --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists-4-2058d464561ef7b24d896ec8ecb21a00 @@ -0,0 +1,11 @@ +98 val_98 +92 val_92 +96 val_96 +95 val_95 +98 val_98 +90 val_90 +95 val_95 +90 val_90 +97 val_97 +90 val_90 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-0-927435f429722c2de003e376b9f0bbd2 b/sql/hive/src/test/resources/golden/subquery_exists_having-0-927435f429722c2de003e376b9f0bbd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-1-b7ac11dbf892c229e180a2bc761117fe b/sql/hive/src/test/resources/golden/subquery_exists_having-1-b7ac11dbf892c229e180a2bc761117fe new file mode 100644 index 0000000000000..3347981aef54b --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists_having-1-b7ac11dbf892c229e180a2bc761117fe @@ -0,0 +1,6 @@ +90 3 +92 1 +95 2 +96 1 +97 2 +98 2 diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-2-4f0b2dbae1324cdc5f3ead83b632e503 b/sql/hive/src/test/resources/golden/subquery_exists_having-2-4f0b2dbae1324cdc5f3ead83b632e503 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-3-da5828589960a60826f5a08948850d78 b/sql/hive/src/test/resources/golden/subquery_exists_having-3-da5828589960a60826f5a08948850d78 new file mode 100644 index 0000000000000..3347981aef54b --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists_having-3-da5828589960a60826f5a08948850d78 @@ -0,0 +1,6 @@ +90 3 +92 1 +95 2 +96 1 +97 2 +98 2 diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-4-fd5457ec549cc2265848f3c95a60693d b/sql/hive/src/test/resources/golden/subquery_exists_having-4-fd5457ec549cc2265848f3c95a60693d new file mode 100644 index 0000000000000..3347981aef54b --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists_having-4-fd5457ec549cc2265848f3c95a60693d @@ -0,0 +1,6 @@ +90 3 +92 1 +95 2 +96 1 +97 2 +98 2 diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-5-aafe13388d5795b26035167edd90a69b b/sql/hive/src/test/resources/golden/subquery_exists_having-5-aafe13388d5795b26035167edd90a69b new file mode 100644 index 0000000000000..6278d429b33b3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists_having-5-aafe13388d5795b26035167edd90a69b @@ -0,0 +1,6 @@ +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-0-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/subquery_in_having-0-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-1-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/subquery_in_having-1-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-10-b8ded52f10f8103684cda7bba20d2201 b/sql/hive/src/test/resources/golden/subquery_in_having-10-b8ded52f10f8103684cda7bba20d2201 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-11-ddeeedb49ded9eb733a4792fff83abe4 b/sql/hive/src/test/resources/golden/subquery_in_having-11-ddeeedb49ded9eb733a4792fff83abe4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-2-877cbfc817ff3718f65073378a0c0829 b/sql/hive/src/test/resources/golden/subquery_in_having-2-877cbfc817ff3718f65073378a0c0829 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-3-63a96439d273b9ad3304d3036bd79e35 b/sql/hive/src/test/resources/golden/subquery_in_having-3-63a96439d273b9ad3304d3036bd79e35 new file mode 100644 index 0000000000000..0f66cd6930d88 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in_having-3-63a96439d273b9ad3304d3036bd79e35 @@ -0,0 +1,303 @@ +10 1 +11 1 +12 2 +15 2 +17 1 +18 2 +19 1 +20 1 +24 2 +26 2 +27 1 +28 1 +30 1 +33 1 +34 1 +35 3 +37 2 +41 1 +42 2 +43 1 +44 1 +47 1 +51 2 +53 1 +54 1 +57 1 +58 2 +64 1 +65 1 +66 1 +67 2 +69 1 +70 3 +72 2 +74 1 +76 2 +77 1 +78 1 +80 1 +82 1 +83 2 +84 2 +85 1 +86 1 +87 1 +90 3 +92 1 +95 2 +96 1 +97 2 +98 2 +100 2 +103 2 +104 2 +105 1 +111 1 +113 2 +114 1 +116 1 +118 2 +119 3 +120 2 +125 2 +126 1 +128 3 +129 2 +131 1 +133 1 +134 2 +136 1 +137 2 +138 4 +143 1 +145 1 +146 2 +149 2 +150 1 +152 2 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 2 +165 2 +166 1 +167 3 +168 1 +169 4 +170 1 +172 2 +174 2 +175 2 +176 2 +177 1 +178 1 +179 2 +180 1 +181 1 +183 1 +186 1 +187 3 +189 1 +190 1 +191 2 +192 1 +193 3 +194 1 +195 2 +196 1 +197 2 +199 3 +200 2 +201 1 +202 1 +203 2 +205 2 +207 2 +208 3 +209 2 +213 2 +214 1 +216 2 +217 2 +218 1 +219 2 +221 2 +222 1 +223 2 +224 2 +226 1 +228 1 +229 2 +230 5 +233 2 +235 1 +237 2 +238 2 +239 2 +241 1 +242 2 +244 1 +247 1 +248 1 +249 1 +252 1 +255 2 +256 2 +257 1 +258 1 +260 1 +262 1 +263 1 +265 2 +266 1 +272 2 +273 3 +274 1 +275 1 +277 4 +278 2 +280 2 +281 2 +282 2 +283 1 +284 1 +285 1 +286 1 +287 1 +288 2 +289 1 +291 1 +292 1 +296 1 +298 3 +302 1 +305 1 +306 1 +307 2 +308 1 +309 2 +310 1 +311 3 +315 1 +316 3 +317 2 +318 3 +321 2 +322 2 +323 1 +325 2 +327 3 +331 2 +332 1 +333 2 +335 1 +336 1 +338 1 +339 1 +341 1 +342 2 +344 2 +345 1 +348 5 +351 1 +353 2 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 2 +368 1 +369 3 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 2 +384 3 +386 1 +389 1 +392 1 +393 1 +394 1 +395 2 +396 3 +397 2 +399 2 +400 1 +401 5 +402 1 +403 3 +404 2 +406 4 +407 1 +409 3 +411 1 +413 2 +414 2 +417 3 +418 1 +419 1 +421 1 +424 2 +427 1 +429 2 +430 3 +431 3 +432 1 +435 1 +436 1 +437 1 +438 3 +439 2 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 3 +455 1 +457 1 +458 2 +459 2 +460 1 +462 2 +463 2 +466 3 +467 1 +468 4 +469 5 +470 1 +472 1 +475 1 +477 1 +478 2 +479 1 +480 3 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 4 +490 1 +491 1 +492 2 +493 1 +494 1 +495 1 +496 1 +497 1 +498 3 diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-4-5d1259d48aa4b26931f1dbe686a0d2d7 b/sql/hive/src/test/resources/golden/subquery_in_having-4-5d1259d48aa4b26931f1dbe686a0d2d7 new file mode 100644 index 0000000000000..52337d4d9809f --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in_having-4-5d1259d48aa4b26931f1dbe686a0d2d7 @@ -0,0 +1,31 @@ +0 3 +5 3 +35 3 +70 3 +90 3 +119 3 +128 3 +167 3 +187 3 +193 3 +199 3 +208 3 +273 3 +298 3 +311 3 +316 3 +318 3 +327 3 +369 3 +384 3 +396 3 +403 3 +409 3 +417 3 +430 3 +431 3 +438 3 +454 3 +466 3 +480 3 +498 3 diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-5-1beb605f3b9b0825c69dc5f52d085225 b/sql/hive/src/test/resources/golden/subquery_in_having-5-1beb605f3b9b0825c69dc5f52d085225 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-6-9543704852a4d71a85b90b85a0c5c0a5 b/sql/hive/src/test/resources/golden/subquery_in_having-6-9543704852a4d71a85b90b85a0c5c0a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-7-6bba00f0273f13733fadbe10b43876f5 b/sql/hive/src/test/resources/golden/subquery_in_having-7-6bba00f0273f13733fadbe10b43876f5 new file mode 100644 index 0000000000000..6278d429b33b3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in_having-7-6bba00f0273f13733fadbe10b43876f5 @@ -0,0 +1,6 @@ +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-8-662f1f7435da5d66fd4b09244387c06b b/sql/hive/src/test/resources/golden/subquery_in_having-8-662f1f7435da5d66fd4b09244387c06b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-9-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/subquery_in_having-9-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in_having-9-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/subquery_notexists-0-75cd3855b33f05667ae76896f4b25d3d b/sql/hive/src/test/resources/golden/subquery_notexists-0-75cd3855b33f05667ae76896f4b25d3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_notexists-1-4ae5bcc868eb27add076db2cb3ca9678 b/sql/hive/src/test/resources/golden/subquery_notexists-1-4ae5bcc868eb27add076db2cb3ca9678 new file mode 100644 index 0000000000000..ce5158c002636 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notexists-1-4ae5bcc868eb27add076db2cb3ca9678 @@ -0,0 +1,119 @@ +165 val_165 +193 val_193 +150 val_150 +128 val_128 +146 val_146 +152 val_152 +145 val_145 +15 val_15 +166 val_166 +153 val_153 +193 val_193 +174 val_174 +199 val_199 +174 val_174 +162 val_162 +167 val_167 +195 val_195 +17 val_17 +113 val_113 +155 val_155 +0 val_0 +128 val_128 +149 val_149 +129 val_129 +170 val_170 +157 val_157 +111 val_111 +169 val_169 +125 val_125 +192 val_192 +187 val_187 +176 val_176 +138 val_138 +103 val_103 +176 val_176 +137 val_137 +180 val_180 +12 val_12 +181 val_181 +138 val_138 +179 val_179 +172 val_172 +129 val_129 +158 val_158 +119 val_119 +0 val_0 +197 val_197 +100 val_100 +199 val_199 +191 val_191 +165 val_165 +120 val_120 +131 val_131 +156 val_156 +196 val_196 +197 val_197 +187 val_187 +137 val_137 +169 val_169 +0 val_0 +179 val_179 +118 val_118 +134 val_134 +138 val_138 +15 val_15 +118 val_118 +19 val_19 +10 val_10 +177 val_177 +11 val_11 +168 val_168 +143 val_143 +160 val_160 +195 val_195 +119 val_119 +149 val_149 +138 val_138 +103 val_103 +113 val_113 +167 val_167 +116 val_116 +191 val_191 +128 val_128 +2 val_2 +193 val_193 +104 val_104 +175 val_175 +105 val_105 +190 val_190 +114 val_114 +12 val_12 +164 val_164 +125 val_125 +164 val_164 +187 val_187 +104 val_104 +163 val_163 +119 val_119 +199 val_199 +120 val_120 +169 val_169 +178 val_178 +136 val_136 +172 val_172 +133 val_133 +175 val_175 +189 val_189 +134 val_134 +18 val_18 +100 val_100 +146 val_146 +186 val_186 +167 val_167 +18 val_18 +183 val_183 +152 val_152 +194 val_194 +126 val_126 +169 val_169 diff --git a/sql/hive/src/test/resources/golden/subquery_notexists-2-73a67f6cae6d8e68efebdab4fbade162 b/sql/hive/src/test/resources/golden/subquery_notexists-2-73a67f6cae6d8e68efebdab4fbade162 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_notexists-3-a8b49a691e12360c7c3fa5df113ba8cf b/sql/hive/src/test/resources/golden/subquery_notexists-3-a8b49a691e12360c7c3fa5df113ba8cf new file mode 100644 index 0000000000000..ce5158c002636 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notexists-3-a8b49a691e12360c7c3fa5df113ba8cf @@ -0,0 +1,119 @@ +165 val_165 +193 val_193 +150 val_150 +128 val_128 +146 val_146 +152 val_152 +145 val_145 +15 val_15 +166 val_166 +153 val_153 +193 val_193 +174 val_174 +199 val_199 +174 val_174 +162 val_162 +167 val_167 +195 val_195 +17 val_17 +113 val_113 +155 val_155 +0 val_0 +128 val_128 +149 val_149 +129 val_129 +170 val_170 +157 val_157 +111 val_111 +169 val_169 +125 val_125 +192 val_192 +187 val_187 +176 val_176 +138 val_138 +103 val_103 +176 val_176 +137 val_137 +180 val_180 +12 val_12 +181 val_181 +138 val_138 +179 val_179 +172 val_172 +129 val_129 +158 val_158 +119 val_119 +0 val_0 +197 val_197 +100 val_100 +199 val_199 +191 val_191 +165 val_165 +120 val_120 +131 val_131 +156 val_156 +196 val_196 +197 val_197 +187 val_187 +137 val_137 +169 val_169 +0 val_0 +179 val_179 +118 val_118 +134 val_134 +138 val_138 +15 val_15 +118 val_118 +19 val_19 +10 val_10 +177 val_177 +11 val_11 +168 val_168 +143 val_143 +160 val_160 +195 val_195 +119 val_119 +149 val_149 +138 val_138 +103 val_103 +113 val_113 +167 val_167 +116 val_116 +191 val_191 +128 val_128 +2 val_2 +193 val_193 +104 val_104 +175 val_175 +105 val_105 +190 val_190 +114 val_114 +12 val_12 +164 val_164 +125 val_125 +164 val_164 +187 val_187 +104 val_104 +163 val_163 +119 val_119 +199 val_199 +120 val_120 +169 val_169 +178 val_178 +136 val_136 +172 val_172 +133 val_133 +175 val_175 +189 val_189 +134 val_134 +18 val_18 +100 val_100 +146 val_146 +186 val_186 +167 val_167 +18 val_18 +183 val_183 +152 val_152 +194 val_194 +126 val_126 +169 val_169 diff --git a/sql/hive/src/test/resources/golden/subquery_notexists_having-0-872612e3ae6ef1445982517a94200075 b/sql/hive/src/test/resources/golden/subquery_notexists_having-0-872612e3ae6ef1445982517a94200075 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_notexists_having-1-8f6c09c8a89cc5939c1c309d660e7b3e b/sql/hive/src/test/resources/golden/subquery_notexists_having-1-8f6c09c8a89cc5939c1c309d660e7b3e new file mode 100644 index 0000000000000..f722855aa13a8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notexists_having-1-8f6c09c8a89cc5939c1c309d660e7b3e @@ -0,0 +1,14 @@ +0 val_0 +10 val_10 +11 val_11 +12 val_12 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 diff --git a/sql/hive/src/test/resources/golden/subquery_notexists_having-2-fb172ff54d6814f42360cb9f30f4882e b/sql/hive/src/test/resources/golden/subquery_notexists_having-2-fb172ff54d6814f42360cb9f30f4882e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_notexists_having-3-edd8e7bbc4bfde58cf744fc0901e2ac b/sql/hive/src/test/resources/golden/subquery_notexists_having-3-edd8e7bbc4bfde58cf744fc0901e2ac new file mode 100644 index 0000000000000..f722855aa13a8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notexists_having-3-edd8e7bbc4bfde58cf744fc0901e2ac @@ -0,0 +1,14 @@ +0 val_0 +10 val_10 +11 val_11 +12 val_12 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 diff --git a/sql/hive/src/test/resources/golden/subquery_notin_having-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/subquery_notin_having-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_notin_having-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/subquery_notin_having-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_notin_having-2-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/subquery_notin_having-2-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_notin_having-3-21a44539fd357dc260687003554fe02a b/sql/hive/src/test/resources/golden/subquery_notin_having-3-21a44539fd357dc260687003554fe02a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subquery_notin_having-4-dea2fabba75cc13e7fa8df072f6b557b b/sql/hive/src/test/resources/golden/subquery_notin_having-4-dea2fabba75cc13e7fa8df072f6b557b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSubquerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSubquerySuite.scala new file mode 100644 index 0000000000000..0d12cd8c8626b --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSubquerySuite.scala @@ -0,0 +1,391 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.{SQLConf, AnalysisException} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.hive.test.TestHive + +/** + * A test suite about the IN /NOT IN /EXISTS / NOT EXISTS subquery. + */ +abstract class HiveSubquerySuite extends HiveComparisonTest with BeforeAndAfter { + import org.apache.spark.sql.hive.test.TestHive.implicits._ + import org.apache.spark.sql.hive.test.TestHive._ + + private val confBroadcastJoin = TestHive.getConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD) + + def enableBroadcastJoin(enable: Boolean): Unit = { + if (enable) { + TestHive.setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, -1) + } else { + TestHive.setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, Int.MaxValue) + } + } + + override def beforeAll() { + // override this method to update the configuration + TestHive.cacheTables = true + } + + override def afterAll() { + // restore the configuration + TestHive.cacheTables = false + } + + ignore("reference the expression `min(b.value)` that required implicit change the outer query") { + sql("""select b.key, min(b.value) + |from src b + |group by b.key + |having exists ( select a.key + |from src a + |where a.value > 'val_9' and a.value = min(b.value))""".stripMargin) + } + + ignore("multiple reference the outer query variables") { + sql("""select key, value, count(*) + |from src b + |group by key, value + |having count(*) in ( + | select count(*) + | from src s1 + | where s1.key > '9' and s1.value = b.value + | group by s1.key)""".stripMargin) + } + + // IN Subquery Unit tests + createQueryTest("(unrelated)WHERE clause with IN #1", + """select * + |from src + |where key in (select key from src) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest("(unrelated)WHERE clause with NOT IN #1", + """select * + |from src + |where key not in (select key from src) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest("(unrelated)WHERE clause with IN #2", + """select * + |from src + |where src.key in (select t.key from src t) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest("(unrelated)WHERE clause with NOT IN #2", + """select * + |from src + |where src.key not in (select t.key % 193 from src t) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with IN #3", + """select * + |from src + |where src.key in (select key from src s1 where s1.key > 9) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with NOT IN #3", + """select * + |from src + |where src.key not in (select key from src s1 where s1.key > 9) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with IN #4", + """select * + |from src + |where src.key in (select max(s1.key) from src s1 group by s1.value) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with NOT IN #4", + """select * + |from src + |where src.key not in (select max(s1.key) % 31 from src s1 group by s1.value) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with IN #5", + """select * + |from src + |where src.key in + |(select max(s1.key) from src s1 group by s1.value having max(s1.key) > 3) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with NOT IN #5", + """select * + |from src + |where src.key not in + |(select max(s1.key) from src s1 group by s1.value having max(s1.key) > 3) + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with IN #6", + """select * + |from src + |where src.key in + |(select max(s1.key) from src s1 group by s1.value having max(s1.key) > 3) + | and src.key > 10 + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with NOT IN #6", + """select * + |from src + |where src.key not in + |(select max(s1.key) % 31 from src s1 group by s1.value having max(s1.key) > 3) + | and src.key > 10 + |order by key, value LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with IN #7", + """select * + | from src b + |where b.key in + |(select count(*) + | from src a + | where a.key > 100 + |) and b.key < 200 + |order by key, value + |LIMIT 5""".stripMargin) + + createQueryTest( + "(unrelated)WHERE clause with NOT IN #7", + """select * + | from src b + |where b.key not in + |(select count(*) + | from src a + | where a.key > 100 + |) and b.key < 200 + |order by key, value + |LIMIT 5""".stripMargin) + + createQueryTest( + "(correlated)WHERE clause with IN #1", + """select * + |from src b + |where b.key in + | (select a.key + | from src a + | where b.value = a.value and a.key > 9 + | ) + |order by key, value + |LIMIT 5""".stripMargin) + + createQueryTest( + "(correlated)WHERE clause with NOT IN #1", + """select * + |from src b + |where b.key not in + | (select a.key + | from src a + | where b.value = a.value and a.key > 9 + | ) + |order by key, value + |LIMIT 5""" + .stripMargin) + + createQueryTest( + "(correlated)WHERE clause with IN #2", + """select * + |from src b + |where b.key in + | (select a.key + | from src a + | where b.value = a.value and a.key > 9 + | ) + |and b.key > 15 + |order by key, value + |LIMIT 5""".stripMargin) + + createQueryTest( + "(correlated)WHERE clause with NOT IN #2", + """select * + |from src b + |where b.key not in + | (select a.key + | from src a + | where b.value = a.value and a.key > 9 + | ) + |and b.key > 15 + |order by key, value + |LIMIT 5""". + stripMargin) + + createQueryTest( + "(correlated)WHERE clause with EXISTS #1", + """select * + |from src b + |where EXISTS + | (select a.key + | from src a + | where b.value = a.value and a.key > 9 + | ) + |order by key, value + |LIMIT 5""". + stripMargin) + + createQueryTest( + "(correlated)WHERE clause with NOT EXISTS #1", + """select * + |from src b + |where NOT EXISTS + | (select a.key + | from src a + | where b.value = a.value and a.key > 9 + | ) + |order by key, value + |LIMIT 5""".stripMargin) + + createQueryTest( + "(correlated)WHERE clause with EXISTS #2", + """select * + |from src b + |where EXISTS + | (select a.key + | from src a + | where b.value = a.value and a.key > 9 + | ) + |and b.key > 15 + |order by key, value + |LIMIT 5""". + stripMargin) + + createQueryTest( + "(correlated)WHERE clause with NOT EXISTS #2", + """select * + |from src b + |where NOT EXISTS + | (select a.key % 291 + | from src a + | where b.value = a.value and a.key > 9 + | ) + |and b.key > 15 + |order by key, value + |LIMIT 5""". + stripMargin) +} + +class SemiJoinHashJoin extends HiveSubquerySuite { + override def beforeAll(): Unit = { + super.beforeAll() + } +} + +class SemiJoinBroadcast extends HiveSubquerySuite { + override def beforeAll(): Unit = { + super.beforeAll() + enableBroadcastJoin(true) + } +} + + +// TODO we should move this to module sql/core, however, the SQLParser doesn't support +// in/exists clause right now. +class SemiJoinSemanticCheckSuite extends HiveComparisonTest with BeforeAndAfter { + import org.apache.spark.sql.hive.test.TestHive._ + + override def beforeAll() { + TestHive.cacheTables = true + } + + override def afterAll() { + TestHive.cacheTables = false + } + + test("semi join key is not in the filter expressions") { + assert(intercept[AnalysisException] { + sql( + """select * + |from src b + |where b.value in + | (select concat("val_", a.key % b.key) + | from src a + | ) + |order by key, value + |LIMIT 5""".stripMargin) + }.getMessage.contains("Outer query expression should be only presented at the filter clause")) + + assert(intercept[AnalysisException] { + sql( + """select * + |from src b + |where b.key in + | (select max(a.key) % b.key + | from src a + | group by a.value + | ) + |order by key, value + |LIMIT 5""".stripMargin) + }.getMessage.contains("Outer query expression should be only presented at the filter clause")) + + assert(intercept[AnalysisException] { + sql( + """select * + |from src b + |where b.key in + | (select max(a.key) % b.key + | from src a + | group by a.value + | having count(a.value) > 2 + | ) + |order by key, value + |LIMIT 5""".stripMargin) + }.getMessage.contains("Outer query expression should be only presented at the filter clause")) + } + + test("Expect only 1 projection in In Subquery Expression") { + assert(intercept[AnalysisException] { + sql( + """select * + |from src b + |where b.key not in + | (select a.key, a.value + | from src a + | where a.key > 9 + | ) + |order by key, value + |LIMIT 5""".stripMargin) + }.getMessage.contains("Expect only 1 projection in In Subquery Expression")) + } + + test("Exist clause should be correlated") { + assert(intercept[AnalysisException] { + sql( + """select * + |from src + |where EXISTS + |(select max(s1.key) from src s1 group by s1.value having max(s1.key) > 3) + |order by key, value LIMIT 5""".stripMargin) + }.getMessage.contains("Exist clause should be correlated")) + + assert(intercept[AnalysisException] { + sql( + """select * + |from src + |where NOT EXISTS + |(select value from src where key > 3) + |order by key, value LIMIT 5""".stripMargin) + }.getMessage.contains("Exist clause should be correlated")) + } +}