[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-11 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r344593154
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
 ##
 @@ -171,6 +171,63 @@ case class InSubqueryExec(
   }
 }
 
+/**
+ * The physical node of exists-subquery. This is for support use exists in 
join's on condition,
+ * since some join type we can't pushdown exists condition, we plan it here
+ */
+case class ExistsExec(child: Expression,
+  subQuery: String,
+  plan: BaseSubqueryExec,
+  exprId: ExprId,
+  private var resultBroadcast: Broadcast[Boolean] = null)
+  extends ExecSubqueryExpression {
+
+  @transient private var result: Boolean = _
+
+  override def dataType: DataType = BooleanType
+  override def children: Seq[Expression] = child :: Nil
+  override def nullable: Boolean = child.nullable
+  override def toString: String = s"EXISTS ${plan.name}"
+  override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = 
plan)
+
+  override def semanticEquals(other: Expression): Boolean = other match {
+case in: ExistsExec => child.semanticEquals(in.child) && 
plan.sameResult(in.plan)
+case _ => false
+  }
+
+
+  def updateResult(): Unit = {
+result = !plan.execute().isEmpty()
+resultBroadcast = plan.sqlContext.sparkContext.broadcast[Boolean](result)
+  }
+
+  def values(): Option[Boolean] = Option(resultBroadcast).map(_.value)
+
+  private def prepareResult(): Unit = {
+require(resultBroadcast != null, s"$this has not finished")
+result = resultBroadcast.value
+  }
+
+  override def eval(input: InternalRow): Any = {
+prepareResult()
+result
+  }
+
+  override lazy val canonicalized: ExistsExec = {
+copy(
+  child = child.canonicalized,
+  subQuery = subQuery,
+  plan = plan.canonicalized.asInstanceOf[BaseSubqueryExec],
+  exprId = ExprId(0),
+  resultBroadcast = null)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+prepareResult()
+ExistsSubquery(child, subQuery, result).doGenCode(ctx, ev)
 
 Review comment:
   > We don't have to extend `UnaryExpression` and we can still implement 
codegen, right?
   
   Done


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-10 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r344589540
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 ##
 @@ -106,12 +106,20 @@ object RewritePredicateSubquery extends 
Rule[LogicalPlan] with PredicateHelper {
 
   // Filter the plan by applying left semi and left anti joins.
   withSubquery.foldLeft(newFilter) {
-case (p, Exists(sub, conditions, _)) =>
-  val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftSemi, joinCond)
-case (p, Not(Exists(sub, conditions, _))) =>
-  val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftAnti, joinCond)
+case (p, exists @ Exists(sub, conditions, _)) =>
+  if (SubqueryExpression.hasCorrelatedSubquery(exists)) {
+val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
+buildJoin(outerPlan, sub, LeftSemi, joinCond)
+  } else {
+Filter(exists, newFilter)
+  }
+case (p, Not(exists @ Exists(sub, conditions, _))) =>
+  if (SubqueryExpression.hasCorrelatedSubquery(exists)) {
+val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
+buildJoin(outerPlan, sub, LeftAnti, joinCond)
+  } else {
+Filter(Not(exists), newFilter)
+  }
 
 Review comment:
   > @AngersZh I discussed this with Wenchen briefly. Do you think we can 
safely inject a "LIMIT 1" into our subplan to expedite its execution ? Pl. lets 
us know what you think ?
   
   I am also thinking about reduce the execution cost of this sub query. 
   `LIMIT 1` is ok . 
   My direction is making this execution  like Spark Thrift Server's 
incremental collect.
   Only execute one partition.
   
   Discuss these two ways safety and cost?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-10 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r344481146
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 ##
 @@ -106,12 +106,20 @@ object RewritePredicateSubquery extends 
Rule[LogicalPlan] with PredicateHelper {
 
   // Filter the plan by applying left semi and left anti joins.
   withSubquery.foldLeft(newFilter) {
-case (p, Exists(sub, conditions, _)) =>
-  val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftSemi, joinCond)
-case (p, Not(Exists(sub, conditions, _))) =>
-  val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftAnti, joinCond)
+case (p, exists @ Exists(sub, conditions, _)) =>
+  if (SubqueryExpression.hasCorrelatedSubquery(exists)) {
+val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
+buildJoin(outerPlan, sub, LeftSemi, joinCond)
+  } else {
+Filter(exists, newFilter)
+  }
+case (p, Not(exists @ Exists(sub, conditions, _))) =>
+  if (SubqueryExpression.hasCorrelatedSubquery(exists)) {
+val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
+buildJoin(outerPlan, sub, LeftAnti, joinCond)
+  } else {
+Filter(Not(exists), newFilter)
+  }
 
 Review comment:
   @cloud-fan @dilipbiswal 
   Change here to support  non-correct exists subquery run like mentioned in 
this comment https://github.com/apache/spark/pull/26437/files#r344203937


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-09 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r33599
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
 ##
 @@ -171,6 +171,63 @@ case class InSubqueryExec(
   }
 }
 
+/**
+ * The physical node of exists-subquery. This is for support use exists in 
join's on condition,
+ * since some join type we can't pushdown exists condition, we plan it here
+ */
+case class ExistsExec(child: Expression,
+  subQuery: String,
+  plan: BaseSubqueryExec,
+  exprId: ExprId,
+  private var resultBroadcast: Broadcast[Boolean] = null)
+  extends ExecSubqueryExpression {
+
+  @transient private var result: Boolean = _
+
+  override def dataType: DataType = BooleanType
+  override def children: Seq[Expression] = child :: Nil
+  override def nullable: Boolean = child.nullable
+  override def toString: String = s"EXISTS ${plan.name}"
+  override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = 
plan)
+
+  override def semanticEquals(other: Expression): Boolean = other match {
+case in: ExistsExec => child.semanticEquals(in.child) && 
plan.sameResult(in.plan)
+case _ => false
+  }
+
+
+  def updateResult(): Unit = {
+result = !plan.execute().isEmpty()
 
 Review comment:
   > Yeah, we need to change RewritePredicateSubquery which handles correlated 
subquery rewrites. The only thing i am not sure is about the outer joins.
   
   Yes, outer join is complex, if we do this, we need to add ent to end test 
case cover each case to make sure the final plans are as expected.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-09 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r33599
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
 ##
 @@ -171,6 +171,63 @@ case class InSubqueryExec(
   }
 }
 
+/**
+ * The physical node of exists-subquery. This is for support use exists in 
join's on condition,
+ * since some join type we can't pushdown exists condition, we plan it here
+ */
+case class ExistsExec(child: Expression,
+  subQuery: String,
+  plan: BaseSubqueryExec,
+  exprId: ExprId,
+  private var resultBroadcast: Broadcast[Boolean] = null)
+  extends ExecSubqueryExpression {
+
+  @transient private var result: Boolean = _
+
+  override def dataType: DataType = BooleanType
+  override def children: Seq[Expression] = child :: Nil
+  override def nullable: Boolean = child.nullable
+  override def toString: String = s"EXISTS ${plan.name}"
+  override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = 
plan)
+
+  override def semanticEquals(other: Expression): Boolean = other match {
+case in: ExistsExec => child.semanticEquals(in.child) && 
plan.sameResult(in.plan)
+case _ => false
+  }
+
+
+  def updateResult(): Unit = {
+result = !plan.execute().isEmpty()
 
 Review comment:
   > Yeah, we need to change RewritePredicateSubquery which handles correlated 
subquery rewrites. The only thing i am not sure is about the outer joins.
   
   Yes, outer join is complex, if we do this, we need to add end to end test 
case cover each case to make sure the final plans are as expected.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-08 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r344426021
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
 ##
 @@ -171,6 +171,63 @@ case class InSubqueryExec(
   }
 }
 
+/**
+ * The physical node of exists-subquery. This is for support use exists in 
join's on condition,
+ * since some join type we can't pushdown exists condition, we plan it here
+ */
+case class ExistsExec(child: Expression,
+  subQuery: String,
+  plan: BaseSubqueryExec,
+  exprId: ExprId,
+  private var resultBroadcast: Broadcast[Boolean] = null)
+  extends ExecSubqueryExpression {
+
+  @transient private var result: Boolean = _
+
+  override def dataType: DataType = BooleanType
+  override def children: Seq[Expression] = child :: Nil
+  override def nullable: Boolean = child.nullable
+  override def toString: String = s"EXISTS ${plan.name}"
+  override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = 
plan)
+
+  override def semanticEquals(other: Expression): Boolean = other match {
+case in: ExistsExec => child.semanticEquals(in.child) && 
plan.sameResult(in.plan)
+case _ => false
+  }
+
+
+  def updateResult(): Unit = {
+result = !plan.execute().isEmpty()
 
 Review comment:
   @dilipbiswal 
   ```
   SELECT * FROM T1 JOIN T2 ON T1.C1 = T2.C1 AND T1.C1 EXISTS (SELECT 1 FROM T3 
WHERE T1.C1 = T3.C1)
   ```
Is not correct .
   
   You mean  below ?
   ```
   SELECT * FROM T1 JOIN T2 ON T1.C1 = T2.C1 AND EXISTS (SELECT 1 FROM T3 WHERE 
T1.C1 = T3.C1)
   ```
   
   For this type sql we need to change `RewritePredicateSubquery` as cloud-fan 
said.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-08 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r344219600
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
 ##
 @@ -171,6 +171,63 @@ case class InSubqueryExec(
   }
 }
 
+/**
+ * The physical node of exists-subquery. This is for support use exists in 
join's on condition,
+ * since some join type we can't pushdown exists condition, we plan it here
+ */
+case class ExistsExec(child: Expression,
+  subQuery: String,
+  plan: BaseSubqueryExec,
+  exprId: ExprId,
+  private var resultBroadcast: Broadcast[Boolean] = null)
+  extends ExecSubqueryExpression {
+
+  @transient private var result: Boolean = _
+
+  override def dataType: DataType = BooleanType
+  override def children: Seq[Expression] = child :: Nil
+  override def nullable: Boolean = child.nullable
+  override def toString: String = s"EXISTS ${plan.name}"
+  override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = 
plan)
+
+  override def semanticEquals(other: Expression): Boolean = other match {
+case in: ExistsExec => child.semanticEquals(in.child) && 
plan.sameResult(in.plan)
+case _ => false
+  }
+
+
+  def updateResult(): Unit = {
+result = !plan.execute().isEmpty()
 
 Review comment:
   > seems like this is better to execute a non-correlated EXISTS subquery. 
Maybe we should update `RewritePredicateSubquery` to only handle correlated 
EXISTS subquery. @dilipbiswal what do you think?
   
   Yeah, wait for his advise.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-08 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r344201008
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
 ##
 @@ -194,6 +257,19 @@ case class PlanSubqueries(sparkSession: SparkSession) 
extends Rule[SparkPlan] {
 }
 val executedPlan = new QueryExecution(sparkSession, query).executedPlan
 InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", 
executedPlan), exprId)
+  case expressions.Exists(sub, children, exprId) =>
 
 Review comment:
   > We can think more about how to solve this problem in your original PR.
   
   Try some ways, result is not the same as PostgresSQL.
   Since for full outer join, we can't push down or build a new join. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-08 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r344200356
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
 ##
 @@ -194,6 +257,19 @@ case class PlanSubqueries(sparkSession: SparkSession) 
extends Rule[SparkPlan] {
 }
 val executedPlan = new QueryExecution(sparkSession, query).executedPlan
 InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", 
executedPlan), exprId)
+  case expressions.Exists(sub, children, exprId) =>
 
 Review comment:
   > We can think more about how to solve this problem in your original PR.
   
   See current change, don't collect data, only judge has result, may don't 
have oom problem


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-08 Thread GitBox
AngersZh commented on a change in pull request #26437: [SPARK-29800][SQL] 
Plan Exists 's subquery in PlanSubqueries
URL: https://github.com/apache/spark/pull/26437#discussion_r344188653
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
 ##
 @@ -171,6 +171,69 @@ case class InSubqueryExec(
   }
 }
 
+/**
+ * The physical node of exists-subquery. This is for support use exists in 
join's on condition,
+ * since some join type we can't pushdown exists condition, we plan it here
+ */
+case class ExistsExec(child: Expression,
+  subQuery: String,
+  plan: BaseSubqueryExec,
+  exprId: ExprId,
+  private var resultBroadcast: Broadcast[Array[Any]] = 
null)
+  extends ExecSubqueryExpression {
+
+  @transient private var result: Array[Any] = _
+
+  override def dataType: DataType = BooleanType
+  override def children: Seq[Expression] = child :: Nil
+  override def nullable: Boolean = child.nullable
+  override def toString: String = s"EXISTS ${plan.name}"
+  override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = 
plan)
+
+  override def semanticEquals(other: Expression): Boolean = other match {
+case in: ExistsExec => child.semanticEquals(in.child) && 
plan.sameResult(in.plan)
+case _ => false
+  }
+
+
+  def updateResult(): Unit = {
+val rows = plan.executeCollect()
 
 Review comment:
   > The reason why we don't have a physical plan for Exists is: it's not 
robust. Collecting the entire result of a query plan at the driver side is very 
likely to hit OOM. That's why we have to convert Exists to a join.
   
   We can make it just return rdd.isEmpy() since exists just need to judge if 
result is empty.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org