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

2019-11-10 Thread GitBox
cloud-fan 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_r344568476
 
 

 ##
 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?


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] cloud-fan commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-08 Thread GitBox
cloud-fan 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_r344205166
 
 

 ##
 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:
   why we create `ExistsSubquery` to only do codegen? can we put the codegen 
logic in `ExistsExec`?


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] cloud-fan commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-08 Thread GitBox
cloud-fan 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_r344203937
 
 

 ##
 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?


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] cloud-fan commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-08 Thread GitBox
cloud-fan 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_r344183324
 
 

 ##
 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 should simply throw exception for any other `SubqueryExpression`, 
explicitly saying that it's not supported.
   
   We can think more about how to solve this problem in your original PR.


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] cloud-fan commented on a change in pull request #26437: [SPARK-29800][SQL] Plan Exists 's subquery in PlanSubqueries

2019-11-08 Thread GitBox
cloud-fan 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_r344182146
 
 

 ##
 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.


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