You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by wzhfy <gi...@git.apache.org> on 2017/03/02 14:30:13 UTC

[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

GitHub user wzhfy opened a pull request:

    https://github.com/apache/spark/pull/17138

    [SPARK-17080] [SQL] join reorder

    ## What changes were proposed in this pull request?
    
    Reorder the joins using a dynamic programming algorithm (Selinger paper):
    First we put all items (basic joined nodes) into level 1, then we build all two-way joins at level 2 from plans at level 1 (single items), then build all 3-way joins from plans at previous levels (two-way joins and single items), then 4-way joins ... etc, until we build all n-way joins and pick the best plan among them.
    
    When building m-way joins, we only keep the best plan (with the lowest cost) for the same set of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among plans (A J B) J C, (A J C) J B and (B J C) J A. Thus, the plans maintained for each level when reordering four items A, B, C, D are as follows:
    ```
    level 1: p({A}), p({B}), p({C}), p({D})
    level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    level 4: p({A, B, C, D})
    ```
    where p({A, B, C, D}) is the final output plan.
    
    For cost evaluation, since physical costs for operators are not available currently, we use cardinalities and sizes to compute costs.
    
    ## How was this patch tested?
    add test cases


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/wzhfy/spark joinReorder

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/17138.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #17138
    
----
commit 4682da4e20327bcf78f979061b9e4366dda25363
Author: wangzhenhua <wa...@huawei.com>
Date:   2017-03-01T08:45:13Z

    join reorder

commit f8b19a81a6a5451150afa618488307c057bde861
Author: wangzhenhua <wa...@huawei.com>
Date:   2017-03-02T14:17:43Z

    add test cases

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104288256
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    --- End diff --
    
    Estimation fails when at least one side (say plan A) of the join don't have rowCount in statistics.
    Then every plan who wants to join with A will fail. Finally there will be no plan in the last level, so the whole procedure of join reorder will also fail.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104125659
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    --- End diff --
    
    Same here: please recurse into the join. Also see the other comment about mutable state.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106773079
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      // Do reordering if the number of items is appropriate and join conditions exist.
    +      // We also need to check if costs of all items can be evaluated.
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty &&
    +          items.forall(_.stats(conf).rowCount.isDefined)) {
    +        JoinReorderDP.search(conf, items, conditions, output).getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set consecutive join nodes ordered.
    +    replaceWithOrderedJoin(result)
    +  }
    +
    +  /**
    +   * Extract consecutive inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = {
    +    plan match {
    +      case Join(left, right, _: InnerLike, cond) =>
    +        val (leftPlans, leftConditions) = extractInnerJoins(left)
    +        val (rightPlans, rightConditions) = extractInnerJoins(right)
    +        (leftPlans ++ rightPlans, cond.toSet.flatMap(splitConjunctivePredicates) ++
    +          leftConditions ++ rightConditions)
    +      case Project(projectList, join) if projectList.forall(_.isInstanceOf[Attribute]) =>
    +        extractInnerJoins(join)
    +      case _ =>
    +        (Seq(plan), Set())
    +    }
    +  }
    +
    +  private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    --- End diff --
    
    Nit: `cond` -> `_`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104760579
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    --- End diff --
    
    @hvanhovell We had a meeting with Sameer and Wenchen on 2/21/2017.  We did not meet you as you were not in San Francisco office on that day.  In the meeting, we agreed to have a good join reorder algorithm implemented in CBO's first release as long as the algorithm has a good reference base.  We can improve the join reorder algorithm later in CBO's second release.  After all, we run short of time for Spark 2.2.  We decided to use the algorithm in Selinger's paper.  For CBO's first release, we will be happy with the join reorder algorithm if it can improve TPC-DS query performance without causing regression.   


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104264589
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala ---
    @@ -60,6 +60,12 @@ trait CatalystConf {
        * Enables CBO for estimation of plan statistics when set true.
        */
       def cboEnabled: Boolean
    +
    +  /** Enables join reorder in CBO. */
    +  def joinReorderEnabled: Boolean
    --- End diff --
    
    shall we use `cboEnabled` for this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104185910
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    --- End diff --
    
    I am not entirely sure we should count on this. It is generally better not to make assumption on the result of rules that should have been running.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106799242
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      // Do reordering if the number of items is appropriate and join conditions exist.
    +      // We also need to check if costs of all items can be evaluated.
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty &&
    +          items.forall(_.stats(conf).rowCount.isDefined)) {
    +        JoinReorderDP.search(conf, items, conditions, output).getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set consecutive join nodes ordered.
    +    replaceWithOrderedJoin(result)
    +  }
    +
    +  /**
    +   * Extract consecutive inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = {
    +    plan match {
    +      case Join(left, right, _: InnerLike, cond) =>
    +        val (leftPlans, leftConditions) = extractInnerJoins(left)
    +        val (rightPlans, rightConditions) = extractInnerJoins(right)
    +        (leftPlans ++ rightPlans, cond.toSet.flatMap(splitConjunctivePredicates) ++
    +          leftConditions ++ rightConditions)
    +      case Project(projectList, join) if projectList.forall(_.isInstanceOf[Attribute]) =>
    +        extractInnerJoins(join)
    +      case _ =>
    +        (Seq(plan), Set())
    +    }
    +  }
    +
    +  private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val replacedLeft = replaceWithOrderedJoin(left)
    +      val replacedRight = replaceWithOrderedJoin(right)
    +      OrderedJoin(j.copy(left = replacedLeft, right = replacedRight))
    +    case p @ Project(_, join) =>
    +      p.copy(child = replaceWithOrderedJoin(join))
    +    case _ =>
    +      plan
    +  }
    +
    +  /** This is a wrapper class for a join node that has been ordered. */
    +  private case class OrderedJoin(join: Join) extends BinaryNode {
    +    override def left: LogicalPlan = join.left
    +    override def right: LogicalPlan = join.right
    +    override def output: Seq[Attribute] = join.output
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm. This implementation is based on the
    + * paper: Access Path Selection in a Relational Database Management System.
    + * http://www.inf.ed.ac.uk/teaching/courses/adbs/AccessPath.pdf
    + *
    + * First we put all items (basic joined nodes) into level 0, then we build all two-way joins
    + * at level 1 from plans at level 0 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 0: p({A}), p({B}), p({C}), p({D})
    + * level 1: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 2: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 3: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +object JoinReorderDP extends PredicateHelper {
    +
    +  def search(
    +      conf: CatalystConf,
    +      items: Seq[LogicalPlan],
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): Option[LogicalPlan] = {
    +
    +    // Level i maintains all found plans for i + 1 items.
    +    // Create the initial plans: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    val foundPlans = mutable.Buffer[JoinPlanMap](itemIndex.map {
    +      case (item, id) => Set(id) -> JoinPlan(Set(id), item, Set(), Cost(0, 0))
    +    }.toMap)
    +
    +    for (lev <- 1 until items.length) {
    +      // Build plans for the next level.
    +      foundPlans += searchLevel(foundPlans, conf, conditions, topOutput)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length - 1)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans at the next level, based on existing levels. */
    +  private def searchLevel(
    +      existingLevels: Seq[JoinPlanMap],
    +      conf: CatalystConf,
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): JoinPlanMap = {
    +
    +    val nextLevel = mutable.Map.empty[Set[Int], JoinPlan]
    +    var k = 0
    +    val lev = existingLevels.length - 1
    +    // Build plans for the next level from plans at level k (one side of the join) and level
    +    // lev - k (the other side of the join).
    +    // For the lower level k, we only need to search from 0 to lev - k, because when building
    +    // a join from A and B, both A J B and B J A are handled.
    +    while (k <= lev - k) {
    +      val oneSideCandidates = existingLevels(k).values.toSeq
    +      for (i <- oneSideCandidates.indices) {
    +        val oneSidePlan = oneSideCandidates(i)
    +        val otherSideCandidates = if (k == lev - k) {
    +          // Both sides of a join are at the same level, no need to repeat for previous ones.
    +          oneSideCandidates.drop(i)
    +        } else {
    +          existingLevels(lev - k).values.toSeq
    +        }
    +
    +        otherSideCandidates.foreach { otherSidePlan =>
    +          // Should not join two overlapping item sets.
    +          if (oneSidePlan.itemIds.intersect(otherSidePlan.itemIds).isEmpty) {
    +            val joinPlan = buildJoin(oneSidePlan, otherSidePlan, conf, conditions, topOutput)
    +            // Check if it's the first plan for the item set, or it's a better plan than
    +            // the existing one due to lower cost.
    +            val existingPlan = nextLevel.get(joinPlan.itemIds)
    +            if (existingPlan.isEmpty || joinPlan.cost.lessThan(existingPlan.get.cost)) {
    +              nextLevel.update(joinPlan.itemIds, joinPlan)
    +            }
    +          }
    +        }
    +      }
    +      k += 1
    +    }
    +    nextLevel.toMap
    +  }
    +
    +  /** Build a new join node. */
    --- End diff --
    
    We need a better description for this function. Also need to document the input parameters and the potential outputs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by nsyca <gi...@git.apache.org>.
Github user nsyca commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    @wzhfy: If the lowest cost plan is the join between {A, B} and {C,D}, can this join reorder algorithm produce this plan? I assume Spark can process bushy join.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104880251
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      // Do reordering if the number of items is appropriate and join conditions exist.
    +      // We also need to check if costs of all items can be evaluated.
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty &&
    +          items.forall(_.stats(conf).rowCount.isDefined)) {
    +        JoinReorderDP.search(conf, items, conditions, output).getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set consecutive join nodes ordered.
    +    replaceWithOrderedJoin(result)
    +  }
    +
    +  /**
    +   * Extract consecutive inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = {
    +    plan match {
    +      case Join(left, right, _: InnerLike, cond) =>
    +        val (leftPlans, leftConditions) = extractInnerJoins(left)
    +        val (rightPlans, rightConditions) = extractInnerJoins(right)
    +        (leftPlans ++ rightPlans, cond.toSet.flatMap(splitConjunctivePredicates) ++
    +          leftConditions ++ rightConditions)
    +      case Project(projectList, join) if projectList.forall(_.isInstanceOf[Attribute]) =>
    +        extractInnerJoins(join)
    +      case _ =>
    +        (Seq(plan), Set())
    +    }
    +  }
    +
    +  private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val replacedLeft = replaceWithOrderedJoin(left)
    +      val replacedRight = replaceWithOrderedJoin(right)
    +      OrderedJoin(j.copy(left = replacedLeft, right = replacedRight))
    +    case p @ Project(_, join) =>
    +      p.copy(child = replaceWithOrderedJoin(join))
    +    case _ =>
    +      plan
    +  }
    +
    +  /** This is a wrapper class for a join node that has been ordered. */
    +  private case class OrderedJoin(join: Join) extends BinaryNode {
    --- End diff --
    
    It is even shorter when you make this a unary node :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104413163
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    --- End diff --
    
    I referred to Selinger's paper: Access Path Selection in a Relational Database Management System.
    The ideas are similar, but I don't know the reference of DPsize.
    Anyway, I'll mention Selinger's paper in the comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104285885
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    --- End diff --
    
    - Join estimation give us output of a LogicalPlan `p`(its size and cardinality), denoted as `T(p)` for simplicity.
    - Cost of the plan is: `Cost(p) = sum(T(join))`, that is, the sum of output of all **intermediate** joins inside the tree of `p`, thus not include leaves or the root of tree. E.g. if p is (A J B) J C, A/B/C are leaves, then cost(p) is just T(A J B), rather than T(p) + T(A J B).
    
    Then, in the above example, suppose A/B/C are all joins, A J B is just another join, so we denote it as D.
    ```
    Costs(CURRENT) = Costs(D) = sum(T(join1)), join1 doesn't contain D
    Costs(OTHER) = Costs(C) = sum(T(join2)), join2 doesn't contain C
    Costs(NEW_JOIN) = Costs(CURRENT) + Costs(OTHER) + T(D) + T(C)
    ```
    So there's no duplication here.
    Maybe it would be less confusing if the code looks like:
    ```
    val cost = Cost(
      curJoinPlan.cost.rows + curPlanStats.rowCount.get + otherJoinPlan.cost.rows + otherPlanStats.rowCount.get, 
      curJoinPlan.cost.size + curPlanStats.size + otherJoinPlan.cost.size + otherPlanStats.size)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104172223
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    --- End diff --
    
    This is a good point. Ideally, choices of join order, join algorithm and enforcer like distribute and sort are made in the same algorithm, but two issues make this difficult to do in Spark.
    1. Catalyst separates logical optimization, physical implementation (strategies) and enforcer (EnsureRequirements) into different stages.
    2. If we want to consider choosing algorithms and enforcers, we need physical cost evaluation to guide us, while we don't have this now.
    So in the current stage, join reordering is logical optimization. In the future after we support physical cost evaluation and maybe modify catalyst's structure, we can have a more mature algorithm. The DP framework will still work, the difference should be the logic of plan cost comparison.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104287124
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -288,6 +288,9 @@ case class Join(
         condition: Option[Expression])
       extends BinaryNode with PredicateHelper {
     
    +  /** Whether this join node is ordered before. Set true after it has been ordered. */
    +  var ordered: Boolean = false
    --- End diff --
    
    Yeah your understanding is right, we could also memonize the joins we have already processed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104281655
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -288,6 +288,9 @@ case class Join(
         condition: Option[Expression])
       extends BinaryNode with PredicateHelper {
     
    +  /** Whether this join node is ordered before. Set true after it has been ordered. */
    +  var ordered: Boolean = false
    --- End diff --
    
    But it's a `transform`, i.e. after we reroder a, b, c, d to a new plan A J B J C J D, we will reorder a, b, c again. Is my understanding right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104403479
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    --- End diff --
    
    Well I thought it might reduce the number of expressions by doing so, but it is probably not worth it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104457984
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    --- End diff --
    
    Thanks for the material. 
    Usually for small number of joins (say less than 12), the current DP algorithm can work well. It has support from both academy (Selinger's paper) and industry (postgresql's standard join reorder). 
    For more complex joins and large number of joins, there will need lots of research and comparison for many kinds of algorithms, e.g. genetic algorithm is used in postgresql.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104128753
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    --- End diff --
    
    This begs the question. Is this the right place to do the reordering? It seems to me that it would be an improvement to do this when we know the distribution and ordering of our the inputs. However the state of the art seems to first do local join reordering, and then optimize for distribution and ordering in a post processing stage.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104281601
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala ---
    @@ -60,6 +60,12 @@ trait CatalystConf {
        * Enables CBO for estimation of plan statistics when set true.
        */
       def cboEnabled: Boolean
    +
    +  /** Enables join reorder in CBO. */
    +  def joinReorderEnabled: Boolean
    --- End diff --
    
    `cboEnabled` controls all things related to stats computation, I think we may need a separate flag for join reorder only.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104171461
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    --- End diff --
    
    NVM you cannot decouple the predicate from the join.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104147006
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    --- End diff --
    
    NIT: Just make one filter condition?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    CBO should be part of planning, right? The result of CBO should be executable plans. IMO, all the CBO rules should be moved the planner. The current optimizer is just rule-based. 
    
    ![screen-shot-2015-04-12-at-8 41 26-am-1024x235](https://cloud.githubusercontent.com/assets/11567269/23736202/80ca67b2-043f-11e7-9aee-c304c8fd0769.png)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106799011
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      // Do reordering if the number of items is appropriate and join conditions exist.
    +      // We also need to check if costs of all items can be evaluated.
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty &&
    +          items.forall(_.stats(conf).rowCount.isDefined)) {
    +        JoinReorderDP.search(conf, items, conditions, output).getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set consecutive join nodes ordered.
    +    replaceWithOrderedJoin(result)
    +  }
    +
    +  /**
    +   * Extract consecutive inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = {
    +    plan match {
    +      case Join(left, right, _: InnerLike, cond) =>
    +        val (leftPlans, leftConditions) = extractInnerJoins(left)
    +        val (rightPlans, rightConditions) = extractInnerJoins(right)
    +        (leftPlans ++ rightPlans, cond.toSet.flatMap(splitConjunctivePredicates) ++
    +          leftConditions ++ rightConditions)
    +      case Project(projectList, join) if projectList.forall(_.isInstanceOf[Attribute]) =>
    +        extractInnerJoins(join)
    +      case _ =>
    +        (Seq(plan), Set())
    +    }
    +  }
    +
    +  private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val replacedLeft = replaceWithOrderedJoin(left)
    +      val replacedRight = replaceWithOrderedJoin(right)
    +      OrderedJoin(j.copy(left = replacedLeft, right = replacedRight))
    +    case p @ Project(_, join) =>
    +      p.copy(child = replaceWithOrderedJoin(join))
    +    case _ =>
    +      plan
    +  }
    +
    +  /** This is a wrapper class for a join node that has been ordered. */
    +  private case class OrderedJoin(join: Join) extends BinaryNode {
    +    override def left: LogicalPlan = join.left
    +    override def right: LogicalPlan = join.right
    +    override def output: Seq[Attribute] = join.output
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm. This implementation is based on the
    + * paper: Access Path Selection in a Relational Database Management System.
    + * http://www.inf.ed.ac.uk/teaching/courses/adbs/AccessPath.pdf
    + *
    + * First we put all items (basic joined nodes) into level 0, then we build all two-way joins
    + * at level 1 from plans at level 0 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 0: p({A}), p({B}), p({C}), p({D})
    + * level 1: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 2: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 3: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +object JoinReorderDP extends PredicateHelper {
    +
    +  def search(
    +      conf: CatalystConf,
    +      items: Seq[LogicalPlan],
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): Option[LogicalPlan] = {
    +
    +    // Level i maintains all found plans for i + 1 items.
    +    // Create the initial plans: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    val foundPlans = mutable.Buffer[JoinPlanMap](itemIndex.map {
    +      case (item, id) => Set(id) -> JoinPlan(Set(id), item, Set(), Cost(0, 0))
    +    }.toMap)
    +
    +    for (lev <- 1 until items.length) {
    +      // Build plans for the next level.
    +      foundPlans += searchLevel(foundPlans, conf, conditions, topOutput)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length - 1)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans at the next level, based on existing levels. */
    +  private def searchLevel(
    +      existingLevels: Seq[JoinPlanMap],
    +      conf: CatalystConf,
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): JoinPlanMap = {
    +
    +    val nextLevel = mutable.Map.empty[Set[Int], JoinPlan]
    +    var k = 0
    +    val lev = existingLevels.length - 1
    +    // Build plans for the next level from plans at level k (one side of the join) and level
    +    // lev - k (the other side of the join).
    +    // For the lower level k, we only need to search from 0 to lev - k, because when building
    +    // a join from A and B, both A J B and B J A are handled.
    +    while (k <= lev - k) {
    +      val oneSideCandidates = existingLevels(k).values.toSeq
    +      for (i <- oneSideCandidates.indices) {
    +        val oneSidePlan = oneSideCandidates(i)
    +        val otherSideCandidates = if (k == lev - k) {
    +          // Both sides of a join are at the same level, no need to repeat for previous ones.
    +          oneSideCandidates.drop(i)
    +        } else {
    +          existingLevels(lev - k).values.toSeq
    +        }
    +
    +        otherSideCandidates.foreach { otherSidePlan =>
    +          // Should not join two overlapping item sets.
    +          if (oneSidePlan.itemIds.intersect(otherSidePlan.itemIds).isEmpty) {
    +            val joinPlan = buildJoin(oneSidePlan, otherSidePlan, conf, conditions, topOutput)
    +            // Check if it's the first plan for the item set, or it's a better plan than
    +            // the existing one due to lower cost.
    +            val existingPlan = nextLevel.get(joinPlan.itemIds)
    +            if (existingPlan.isEmpty || joinPlan.cost.lessThan(existingPlan.get.cost)) {
    +              nextLevel.update(joinPlan.itemIds, joinPlan)
    +            }
    +          }
    +        }
    +      }
    +      k += 1
    +    }
    +    nextLevel.toMap
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(
    +      oneJoinPlan: JoinPlan,
    +      otherJoinPlan: JoinPlan,
    +      conf: CatalystConf,
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): JoinPlan = {
    +
    +    val onePlan = oneJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    // Now both onePlan and otherPlan become intermediate joins, so the cost of the
    +    // new join should also include their own cardinalities and sizes.
    +    val newCost = if (isCartesianProduct(onePlan) || isCartesianProduct(otherPlan)) {
    +      // We consider cartesian product very expensive, thus set a very large cost for it.
    +      // This enables to plan all the cartesian products at the end, because having a cartesian
    +      // product as an intermediate join will significantly increase a plan's cost, making it
    +      // impossible to be selected as the best plan for the items, unless there's no other choice.
    +      Cost(
    +        rows = BigInt(Long.MaxValue) * BigInt(Long.MaxValue),
    +        size = BigInt(Long.MaxValue) * BigInt(Long.MaxValue))
    +    } else {
    +      val onePlanStats = onePlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      Cost(
    +        rows = oneJoinPlan.cost.rows + onePlanStats.rowCount.get +
    +          otherJoinPlan.cost.rows + otherPlanStats.rowCount.get,
    +        size = oneJoinPlan.cost.size + onePlanStats.sizeInBytes +
    +          otherJoinPlan.cost.size + otherPlanStats.sizeInBytes)
    +    }
    +
    +    // Put the deeper side on the left, tend to build a left-deep tree.
    +    val (left, right) = if (oneJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +      (onePlan, otherPlan)
    +    } else {
    +      (otherPlan, onePlan)
    +    }
    +    val joinConds = conditions
    +      .filterNot(l => canEvaluate(l, onePlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(onePlan.outputSet ++ otherPlan.outputSet))
    +    // We use inner join whether join condition is empty or not. Since cross join is
    +    // equivalent to inner join without condition.
    +    val newJoin = Join(left, right, Inner, joinConds.reduceOption(And))
    +    val collectedJoinConds = joinConds ++ oneJoinPlan.joinConds ++ otherJoinPlan.joinConds
    +    val remainingConds = conditions -- collectedJoinConds
    +    val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +    val neededFromNewJoin = newJoin.outputSet.filter(neededAttr.contains)
    +    val newPlan =
    +      if ((newJoin.outputSet -- neededFromNewJoin).nonEmpty) {
    +        Project(neededFromNewJoin.toSeq, newJoin)
    +      } else {
    +        newJoin
    +      }
    +
    +    val itemIds = oneJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +    JoinPlan(itemIds, newPlan, collectedJoinConds, newCost)
    +  }
    +
    +  private def isCartesianProduct(plan: LogicalPlan): Boolean = plan match {
    +    case Join(_, _, _, None) => true
    +    case Project(_, Join(_, _, _, None)) => true
    +    case _ => false
    +  }
    +
    +  /** Map[set of item ids, join plan for these items] */
    +  type JoinPlanMap = Map[Set[Int], JoinPlan]
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param joinConds Join conditions included in the plan.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, joinConds: Set[Expression], cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    --- End diff --
    
    This is pretty important class. Could you add @param for both parameters? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    also cc @ron8hu 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104135795
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    --- End diff --
    
    I am not really sure you need this. Just internalize this in the `search` and `searchForLevel` functions. That will also allow you to turn this into an object.
    
    BTW, if ever need to do something like this again. Use: `Array.fill(items.length)(mutable.Map.empty)` 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Spark can process bushy joins. This implementation of `DPSize` should be able to create bushy joins (or I have missed something).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104880245
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      // Do reordering if the number of items is appropriate and join conditions exist.
    +      // We also need to check if costs of all items can be evaluated.
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty &&
    +          items.forall(_.stats(conf).rowCount.isDefined)) {
    +        JoinReorderDP.search(conf, items, conditions, output).getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set consecutive join nodes ordered.
    +    replaceWithOrderedJoin(result)
    +  }
    +
    +  /**
    +   * Extract consecutive inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = {
    +    plan match {
    +      case Join(left, right, _: InnerLike, cond) =>
    +        val (leftPlans, leftConditions) = extractInnerJoins(left)
    +        val (rightPlans, rightConditions) = extractInnerJoins(right)
    +        (leftPlans ++ rightPlans, cond.toSet.flatMap(splitConjunctivePredicates) ++
    +          leftConditions ++ rightConditions)
    +      case Project(projectList, join) if projectList.forall(_.isInstanceOf[Attribute]) =>
    +        extractInnerJoins(join)
    +      case _ =>
    +        (Seq(plan), Set())
    +    }
    +  }
    +
    +  private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val replacedLeft = replaceWithOrderedJoin(left)
    +      val replacedRight = replaceWithOrderedJoin(right)
    +      OrderedJoin(j.copy(left = replacedLeft, right = replacedRight))
    +    case p @ Project(_, join) =>
    --- End diff --
    
    NIT: You can make this pattern match a bit more precise by also checking if `join` is an actual `Join`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104125535
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    --- End diff --
    
    Can you just recurse into the join? There is no point in repeating the code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by nsyca <gi...@git.apache.org>.
Github user nsyca commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    I'd start with my definition of a cost-based optimizer (CBO). Cost-based optimizer is a process where an optimal execution plan is chosen based on its estimated cost of execution. The estimated cost can be as crude as (1) a function of the estimated number of output rows from each operation in the plan, or as sophisticated as (2) modelling the execution cost of each operator.
    
    Cardinality estimation is a foundation of a CBO but if we really want to model the execution cost, it will need to calculate at the point we pick an execution method, such as what join methods are eligible for such logical operation, either a hash-based aggregation or a sort-based aggregation; either a hash join, a sorted merge join, or a nested loop join; in the case of a join, which table is on the inner, which on the outer; whether we want to reshuffle the input or broadcast the input. It can go even further that which properties of the output are desired properties of the output that may be beneficial to the next operator.
    
    An example is
    
     (T1 join T2 on T1.c1 = T2.c1 join T3 on T1.c2 = T3.c2) group by T1.c1
    
    Do we want to keep the plan for the joins of {T1, T2, T3} so that it will preserve the order of T1.c1 so that the subsequent aggregate comes virtually free? Or will we just pick the lowest cost of joins but lose the order and then do a hash aggregate?
    
    If the goal of Spark is to do (2) eventually, it is prudent to lay a good foundation by doing any CBO functionality in the Physical Planning phase.
    
    On a slightly off topic, cardinality estimate is useful even in rule-based optimization in Logical Optimization. Rule-based optimization can do a better if it has access to cardinality estimation, especially in some advanced optimization rules.
    
    An example is
    
    (small_tab T1 left join big_tab T2 on T1.c1 = T2.c1) group by T2.c1
    
    if we know that T1.c1 from the small_tab will filter a lot of rows from big_tab, then a desired rewritten query would be
    
    <some compensation> (  ( small_tab T1 left join big_tab T2 ) group by T2.c1 )



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104175341
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case _ =>
    +      Set()
    +  }
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, sizeInBytes: BigInt) {
    --- End diff --
    
    Yes, in the future we could have a more comprehensive model including physical costs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106775288
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    --- End diff --
    
    Are we excluding non-leaf nodes here? It could be any sub-plan, right? For example, an outer join node could be returned as items?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104130473
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    --- End diff --
    
    How are you dealing with projects that create join columns, or columns that are used in the result?
    
    At this stage of optimization it is relatively safe to assume that projects are there for a reason.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104420290
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    --- End diff --
    
    The following slide deck mentions `DPSize`: http://dsg.uwaterloo.ca/seminars/notes/Guido.pdf
    
    They seem to favor a more advanced DP approach called `DPccp`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104864057
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    --- End diff --
    
    why do we have this case?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104285948
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    --- End diff --
    
    When join estimation for `curPlan` and `otherPlan` both succeed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/17138


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104883388
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    --- End diff --
    
    NIT: I assume that you want to transform down the tree? If you do it is better to use `transformDown`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104346324
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    --- End diff --
    
    Sorry can you explain a little more here? Why should I canonicalize?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106773013
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      // Do reordering if the number of items is appropriate and join conditions exist.
    +      // We also need to check if costs of all items can be evaluated.
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty &&
    +          items.forall(_.stats(conf).rowCount.isDefined)) {
    +        JoinReorderDP.search(conf, items, conditions, output).getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set consecutive join nodes ordered.
    +    replaceWithOrderedJoin(result)
    +  }
    +
    +  /**
    +   * Extract consecutive inner joinable items and join conditions.
    --- End diff --
    
    How about
    
    `Extracts the join conditions and sub-plans of consecutive inner joins`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104153377
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    --- End diff --
    
    Let's move `newJoin.outputSet.filter(neededAttr.contains)` into a separate variable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104346049
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    --- End diff --
    
    My goal is dealing with only consecutive joins, but due to column pruning, there could be Projects that break join chains. So I only want to support joins and projects from column pruning here.
    But I didn't realize other projects like you mentioned above.
    Then what about we recurse only when the project list are all attributes (not compound expression like `a+b`)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104175917
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case _ =>
    +      Set()
    +  }
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, sizeInBytes: BigInt) {
    +  /**
    +   * An empirical value for the weights of cardinality (number of rows) in the cost formula:
    +   * cost = rows * weight + size * (1 - weight), usually cardinality is more important than size.
    +   */
    +  val weight = 0.7
    +
    +  def +(other: Cost): Cost = Cost(rows + other.rows, sizeInBytes + other.sizeInBytes)
    --- End diff --
    
    Sorry can you give me a clue about the style? Is it about the method name? I see `BitSet.scala` has something similar like `def &(other: BitSet): BitSet`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104185278
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    --- End diff --
    
    But that means you are counting things twice right? Calling `LogicalPlan.stats(conf)` recursively determines costs for a `LogicalPlan`. The `JoinPlan` contains costs for the children of the join or 0 if there is no join. So if you combine those things seem to the duplicated, for example:
    ```
    CURRENT = (TABLE A) JOIN (TABLE B) => Cost = c_A + c_B
    OTHER = (TABLE C)
    JOIN = ((TABLE A) JOIN (TABLE B) JOIN (TABLE C))
    ```
    This given the following cost structure (`J` is the join statistics estimation function):
    ```
    Costs(CURRENT) =  c_A + c_B
    Costs(OTHER) = c_C
    Costs(JOIN) = Costs(Current) + Cost(Other) + Cost((TABLE A) JOIN (TABLE B)) + Cost((TABLE C))
             = c_A + c_B + J(c_A, c_B) + c_C
    
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104142173
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    --- End diff --
    
    What is happening to index `0`? Are you skipping this on purpose?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104372768
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    --- End diff --
    
    Shouldn't try to fail early here? We should certainly check if we can fail early when we cannot do this for one of the inputs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    **[Test build #74207 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74207/testReport)** for PR 17138 at commit [`757e3b6`](https://github.com/apache/spark/commit/757e3b65083e860aba3d1161c914c547545a28f5).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104146320
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    --- End diff --
    
    Doesn't this mean that the algorithm might not converge as as soon as we have a cross join in there? We should either just plan the cross joins at the end, or return a result for them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104335968
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case _ =>
    +      Set()
    +  }
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, sizeInBytes: BigInt) {
    +  /**
    +   * An empirical value for the weights of cardinality (number of rows) in the cost formula:
    +   * cost = rows * weight + size * (1 - weight), usually cardinality is more important than size.
    +   */
    +  val weight = 0.7
    +
    +  def +(other: Cost): Cost = Cost(rows + other.rows, sizeInBytes + other.sizeInBytes)
    --- End diff --
    
    In databricks scala style, it says:
    
    > Do NOT use symbolic method names, unless you are defining them for natural arithmetic operations (e.g. +, -, *, /). 
    
    Here both `+` and `<` are natural arithmetic operations, so I think they are ok.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104125352
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -288,6 +288,9 @@ case class Join(
         condition: Option[Expression])
       extends BinaryNode with PredicateHelper {
     
    +  /** Whether this join node is ordered before. Set true after it has been ordered. */
    +  var ordered: Boolean = false
    --- End diff --
    
    Could you try to avoid mutable state. Either add the boolean a parameter, or wrap the join.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    **[Test build #73767 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73767/testReport)** for PR 17138 at commit [`f8b19a8`](https://github.com/apache/spark/commit/f8b19a81a6a5451150afa618488307c057bde861).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104372879
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case _ =>
    +      Set()
    +  }
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, sizeInBytes: BigInt) {
    +  /**
    +   * An empirical value for the weights of cardinality (number of rows) in the cost formula:
    +   * cost = rows * weight + size * (1 - weight), usually cardinality is more important than size.
    +   */
    +  val weight = 0.7
    +
    +  def +(other: Cost): Cost = Cost(rows + other.rows, sizeInBytes + other.sizeInBytes)
    --- End diff --
    
    Ok good :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106783976
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    --- End diff --
    
    `subplan` is a very general term. It is like sub-trees. `Items` does not convey the semantics you want. How about `inputPlans`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106798982
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -668,6 +668,18 @@ object SQLConf {
           .booleanConf
           .createWithDefault(false)
     
    +  val JOIN_REORDER_ENABLED =
    +    buildConf("spark.sql.cbo.joinReorder.enabled")
    +      .doc("Enables join reorder in CBO.")
    +      .booleanConf
    +      .createWithDefault(false)
    +
    +  val JOIN_REORDER_DP_THRESHOLD =
    +    buildConf("spark.sql.cbo.joinReorder.dp.threshold")
    +      .doc("The maximum number of joined nodes allowed in the dynamic programming algorithm.")
    +      .intConf
    +      .createWithDefault(12)
    --- End diff --
    
    Also add a conf value check here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104281605
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala ---
    @@ -60,6 +60,12 @@ trait CatalystConf {
        * Enables CBO for estimation of plan statistics when set true.
        */
       def cboEnabled: Boolean
    +
    +  /** Enables join reorder in CBO. */
    +  def joinReorderEnabled: Boolean
    --- End diff --
    
    `cboEnabled` controls all things related to stats computation, I think we may need a separate flag for join reorder only.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106775194
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      // Do reordering if the number of items is appropriate and join conditions exist.
    +      // We also need to check if costs of all items can be evaluated.
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty &&
    +          items.forall(_.stats(conf).rowCount.isDefined)) {
    +        JoinReorderDP.search(conf, items, conditions, output).getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set consecutive join nodes ordered.
    +    replaceWithOrderedJoin(result)
    +  }
    +
    +  /**
    +   * Extract consecutive inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = {
    +    plan match {
    +      case Join(left, right, _: InnerLike, cond) =>
    +        val (leftPlans, leftConditions) = extractInnerJoins(left)
    +        val (rightPlans, rightConditions) = extractInnerJoins(right)
    +        (leftPlans ++ rightPlans, cond.toSet.flatMap(splitConjunctivePredicates) ++
    +          leftConditions ++ rightConditions)
    +      case Project(projectList, join) if projectList.forall(_.isInstanceOf[Attribute]) =>
    +        extractInnerJoins(join)
    +      case _ =>
    +        (Seq(plan), Set())
    +    }
    +  }
    +
    +  private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val replacedLeft = replaceWithOrderedJoin(left)
    +      val replacedRight = replaceWithOrderedJoin(right)
    +      OrderedJoin(j.copy(left = replacedLeft, right = replacedRight))
    +    case p @ Project(_, join) =>
    +      p.copy(child = replaceWithOrderedJoin(join))
    +    case _ =>
    +      plan
    +  }
    +
    +  /** This is a wrapper class for a join node that has been ordered. */
    +  private case class OrderedJoin(join: Join) extends BinaryNode {
    +    override def left: LogicalPlan = join.left
    +    override def right: LogicalPlan = join.right
    +    override def output: Seq[Attribute] = join.output
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm. This implementation is based on the
    + * paper: Access Path Selection in a Relational Database Management System.
    + * http://www.inf.ed.ac.uk/teaching/courses/adbs/AccessPath.pdf
    + *
    + * First we put all items (basic joined nodes) into level 0, then we build all two-way joins
    + * at level 1 from plans at level 0 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 0: p({A}), p({B}), p({C}), p({D})
    + * level 1: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 2: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 3: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +object JoinReorderDP extends PredicateHelper {
    +
    +  def search(
    +      conf: CatalystConf,
    +      items: Seq[LogicalPlan],
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): Option[LogicalPlan] = {
    +
    +    // Level i maintains all found plans for i + 1 items.
    +    // Create the initial plans: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    val foundPlans = mutable.Buffer[JoinPlanMap](itemIndex.map {
    +      case (item, id) => Set(id) -> JoinPlan(Set(id), item, Set(), Cost(0, 0))
    +    }.toMap)
    +
    +    for (lev <- 1 until items.length) {
    +      // Build plans for the next level.
    +      foundPlans += searchLevel(foundPlans, conf, conditions, topOutput)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length - 1)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans at the next level, based on existing levels. */
    +  private def searchLevel(
    +      existingLevels: Seq[JoinPlanMap],
    +      conf: CatalystConf,
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): JoinPlanMap = {
    +
    +    val nextLevel = mutable.Map.empty[Set[Int], JoinPlan]
    +    var k = 0
    +    val lev = existingLevels.length - 1
    +    // Build plans for the next level from plans at level k (one side of the join) and level
    +    // lev - k (the other side of the join).
    +    // For the lower level k, we only need to search from 0 to lev - k, because when building
    +    // a join from A and B, both A J B and B J A are handled.
    +    while (k <= lev - k) {
    +      val oneSideCandidates = existingLevels(k).values.toSeq
    +      for (i <- oneSideCandidates.indices) {
    +        val oneSidePlan = oneSideCandidates(i)
    +        val otherSideCandidates = if (k == lev - k) {
    +          // Both sides of a join are at the same level, no need to repeat for previous ones.
    +          oneSideCandidates.drop(i)
    +        } else {
    +          existingLevels(lev - k).values.toSeq
    +        }
    +
    +        otherSideCandidates.foreach { otherSidePlan =>
    +          // Should not join two overlapping item sets.
    +          if (oneSidePlan.itemIds.intersect(otherSidePlan.itemIds).isEmpty) {
    +            val joinPlan = buildJoin(oneSidePlan, otherSidePlan, conf, conditions, topOutput)
    +            // Check if it's the first plan for the item set, or it's a better plan than
    +            // the existing one due to lower cost.
    +            val existingPlan = nextLevel.get(joinPlan.itemIds)
    +            if (existingPlan.isEmpty || joinPlan.cost.lessThan(existingPlan.get.cost)) {
    +              nextLevel.update(joinPlan.itemIds, joinPlan)
    +            }
    +          }
    +        }
    +      }
    +      k += 1
    +    }
    +    nextLevel.toMap
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(
    +      oneJoinPlan: JoinPlan,
    +      otherJoinPlan: JoinPlan,
    +      conf: CatalystConf,
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): JoinPlan = {
    +
    +    val onePlan = oneJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    // Now both onePlan and otherPlan become intermediate joins, so the cost of the
    +    // new join should also include their own cardinalities and sizes.
    +    val newCost = if (isCartesianProduct(onePlan) || isCartesianProduct(otherPlan)) {
    +      // We consider cartesian product very expensive, thus set a very large cost for it.
    +      // This enables to plan all the cartesian products at the end, because having a cartesian
    +      // product as an intermediate join will significantly increase a plan's cost, making it
    +      // impossible to be selected as the best plan for the items, unless there's no other choice.
    +      Cost(
    +        rows = BigInt(Long.MaxValue) * BigInt(Long.MaxValue),
    +        size = BigInt(Long.MaxValue) * BigInt(Long.MaxValue))
    +    } else {
    +      val onePlanStats = onePlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      Cost(
    +        rows = oneJoinPlan.cost.rows + onePlanStats.rowCount.get +
    +          otherJoinPlan.cost.rows + otherPlanStats.rowCount.get,
    +        size = oneJoinPlan.cost.size + onePlanStats.sizeInBytes +
    +          otherJoinPlan.cost.size + otherPlanStats.sizeInBytes)
    +    }
    +
    +    // Put the deeper side on the left, tend to build a left-deep tree.
    +    val (left, right) = if (oneJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +      (onePlan, otherPlan)
    +    } else {
    +      (otherPlan, onePlan)
    +    }
    +    val joinConds = conditions
    +      .filterNot(l => canEvaluate(l, onePlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(onePlan.outputSet ++ otherPlan.outputSet))
    +    // We use inner join whether join condition is empty or not. Since cross join is
    +    // equivalent to inner join without condition.
    +    val newJoin = Join(left, right, Inner, joinConds.reduceOption(And))
    +    val collectedJoinConds = joinConds ++ oneJoinPlan.joinConds ++ otherJoinPlan.joinConds
    +    val remainingConds = conditions -- collectedJoinConds
    +    val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +    val neededFromNewJoin = newJoin.outputSet.filter(neededAttr.contains)
    +    val newPlan =
    +      if ((newJoin.outputSet -- neededFromNewJoin).nonEmpty) {
    +        Project(neededFromNewJoin.toSeq, newJoin)
    +      } else {
    +        newJoin
    +      }
    +
    +    val itemIds = oneJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +    JoinPlan(itemIds, newPlan, collectedJoinConds, newCost)
    +  }
    +
    +  private def isCartesianProduct(plan: LogicalPlan): Boolean = plan match {
    +    case Join(_, _, _, None) => true
    +    case Project(_, Join(_, _, _, None)) => true
    +    case _ => false
    +  }
    +
    +  /** Map[set of item ids, join plan for these items] */
    +  type JoinPlanMap = Map[Set[Int], JoinPlan]
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param joinConds Join conditions included in the plan.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, joinConds: Set[Expression], cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, size: BigInt) {
    --- End diff --
    
    ok, I'll change it in a follow up, thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104146550
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case _ =>
    +      Set()
    +  }
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, sizeInBytes: BigInt) {
    --- End diff --
    
    Do we need a separate `Cost` class? Or are we planning to use different cost models?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    **[Test build #73767 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73767/testReport)** for PR 17138 at commit [`f8b19a8`](https://github.com/apache/spark/commit/f8b19a81a6a5451150afa618488307c057bde861).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106773023
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    --- End diff --
    
    Nit: `items` -> `subplans`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104129596
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    --- End diff --
    
    Nit - it is a little bit shorter to write: `cond.toSet.flatMap(splitConjunctivePredicates)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104888422
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.optimizer
    +
    +import org.apache.spark.sql.catalyst.SimpleCatalystConf
    +import org.apache.spark.sql.catalyst.dsl.expressions._
    +import org.apache.spark.sql.catalyst.dsl.plans._
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
    +import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest}
    +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, LogicalPlan}
    +import org.apache.spark.sql.catalyst.rules.RuleExecutor
    +import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan}
    +import org.apache.spark.sql.catalyst.util._
    +
    +
    +class JoinReorderSuite extends PlanTest with StatsEstimationTestBase {
    --- End diff --
    
    Could you also add a test case for cross joins?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104266948
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case _ =>
    +      Set()
    +  }
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, sizeInBytes: BigInt) {
    +  /**
    +   * An empirical value for the weights of cardinality (number of rows) in the cost formula:
    +   * cost = rows * weight + size * (1 - weight), usually cardinality is more important than size.
    +   */
    +  val weight = 0.7
    +
    +  def +(other: Cost): Cost = Cost(rows + other.rows, sizeInBytes + other.sizeInBytes)
    --- End diff --
    
    `def +` should be OK, but `def <` looks weird as it's not symmetric. How about we add `def value: Double` and use `cost1.value < cost2.value`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104139355
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    --- End diff --
    
    This is a very complex way of writing: `(i <- 0 until joinPlansLevelK.size) { }`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104174772
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    --- End diff --
    
    JoinEstimation estimates the output size and cardinality of a join node. Here the cost of a JoinPlan is the sum of costs of all intermediate joins.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104281554
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case _ =>
    +      Set()
    +  }
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, sizeInBytes: BigInt) {
    +  /**
    +   * An empirical value for the weights of cardinality (number of rows) in the cost formula:
    +   * cost = rows * weight + size * (1 - weight), usually cardinality is more important than size.
    +   */
    +  val weight = 0.7
    +
    +  def +(other: Cost): Cost = Cost(rows + other.rows, sizeInBytes + other.sizeInBytes)
    --- End diff --
    
    We need to compute a relative value inside the comparison logic, how about change the name to `def lt` or `def lessThan`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104142098
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    --- End diff --
    
    It took me a while to understand that you are writing to this. Please reorganize this into a function that takes the previous results and generates new result. For example:
    ```scala
    type JoinPlanMap = Map[Set[Int], JoinPlan]
    private def searchLevel(previous: Seq[JoinPlanMap]): JoinPlanMap = {
      val result = mutable.Map.empty[JoinPlanMap]
      val level = previous.size
      var k = 0
      while (k <= level - k) {
        // Current code
        ...
        k += 1
      }
      result
    }
    ```
    and then you use the following in `search`:
    ```scala
    // Create the initial plans: each plan is a single item with zero cost.
    val foundPlans = mutable.Buffer[JoinPlanMap](itemIndex.map {
      case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0))
    })
    
    // Build the next plans.
    var level = 1
    while (i < items.length) {
      foundPlans += searchLevel(level)
      level += 1
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104287149
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case _ =>
    +      Set()
    +  }
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, sizeInBytes: BigInt) {
    +  /**
    +   * An empirical value for the weights of cardinality (number of rows) in the cost formula:
    +   * cost = rows * weight + size * (1 - weight), usually cardinality is more important than size.
    +   */
    +  val weight = 0.7
    +
    +  def +(other: Cost): Cost = Cost(rows + other.rows, sizeInBytes + other.sizeInBytes)
    --- End diff --
    
    I am just talking about overloading symbols. I suppose this is fine for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104865669
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    --- End diff --
    
    1. It's used before filter pushdown and column pruning, so the tree structure are different from that in our case.
    2. It only supports left-deep tree.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104372330
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    --- End diff --
    
    Could you mention which paper this comes from and the name algorithm you are using? The name of the algorithm is `DPsize` right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104287137
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    --- End diff --
    
    I personally think it is more confusing if you start at `1`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    **[Test build #73772 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73772/testReport)** for PR 17138 at commit [`cb35885`](https://github.com/apache/spark/commit/cb35885873c81638ea116c874576787c030335cb).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106774778
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      // Do reordering if the number of items is appropriate and join conditions exist.
    +      // We also need to check if costs of all items can be evaluated.
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty &&
    +          items.forall(_.stats(conf).rowCount.isDefined)) {
    +        JoinReorderDP.search(conf, items, conditions, output).getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set consecutive join nodes ordered.
    +    replaceWithOrderedJoin(result)
    +  }
    +
    +  /**
    +   * Extract consecutive inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = {
    +    plan match {
    +      case Join(left, right, _: InnerLike, cond) =>
    +        val (leftPlans, leftConditions) = extractInnerJoins(left)
    +        val (rightPlans, rightConditions) = extractInnerJoins(right)
    +        (leftPlans ++ rightPlans, cond.toSet.flatMap(splitConjunctivePredicates) ++
    +          leftConditions ++ rightConditions)
    +      case Project(projectList, join) if projectList.forall(_.isInstanceOf[Attribute]) =>
    +        extractInnerJoins(join)
    +      case _ =>
    +        (Seq(plan), Set())
    +    }
    +  }
    +
    +  private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val replacedLeft = replaceWithOrderedJoin(left)
    +      val replacedRight = replaceWithOrderedJoin(right)
    +      OrderedJoin(j.copy(left = replacedLeft, right = replacedRight))
    +    case p @ Project(_, join) =>
    +      p.copy(child = replaceWithOrderedJoin(join))
    +    case _ =>
    +      plan
    +  }
    +
    +  /** This is a wrapper class for a join node that has been ordered. */
    +  private case class OrderedJoin(join: Join) extends BinaryNode {
    +    override def left: LogicalPlan = join.left
    +    override def right: LogicalPlan = join.right
    +    override def output: Seq[Attribute] = join.output
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm. This implementation is based on the
    + * paper: Access Path Selection in a Relational Database Management System.
    + * http://www.inf.ed.ac.uk/teaching/courses/adbs/AccessPath.pdf
    + *
    + * First we put all items (basic joined nodes) into level 0, then we build all two-way joins
    + * at level 1 from plans at level 0 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 0: p({A}), p({B}), p({C}), p({D})
    + * level 1: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 2: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 3: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +object JoinReorderDP extends PredicateHelper {
    +
    +  def search(
    +      conf: CatalystConf,
    +      items: Seq[LogicalPlan],
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): Option[LogicalPlan] = {
    +
    +    // Level i maintains all found plans for i + 1 items.
    +    // Create the initial plans: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    val foundPlans = mutable.Buffer[JoinPlanMap](itemIndex.map {
    +      case (item, id) => Set(id) -> JoinPlan(Set(id), item, Set(), Cost(0, 0))
    +    }.toMap)
    +
    +    for (lev <- 1 until items.length) {
    +      // Build plans for the next level.
    +      foundPlans += searchLevel(foundPlans, conf, conditions, topOutput)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length - 1)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans at the next level, based on existing levels. */
    +  private def searchLevel(
    +      existingLevels: Seq[JoinPlanMap],
    +      conf: CatalystConf,
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): JoinPlanMap = {
    +
    +    val nextLevel = mutable.Map.empty[Set[Int], JoinPlan]
    +    var k = 0
    +    val lev = existingLevels.length - 1
    +    // Build plans for the next level from plans at level k (one side of the join) and level
    +    // lev - k (the other side of the join).
    +    // For the lower level k, we only need to search from 0 to lev - k, because when building
    +    // a join from A and B, both A J B and B J A are handled.
    +    while (k <= lev - k) {
    +      val oneSideCandidates = existingLevels(k).values.toSeq
    +      for (i <- oneSideCandidates.indices) {
    +        val oneSidePlan = oneSideCandidates(i)
    +        val otherSideCandidates = if (k == lev - k) {
    +          // Both sides of a join are at the same level, no need to repeat for previous ones.
    +          oneSideCandidates.drop(i)
    +        } else {
    +          existingLevels(lev - k).values.toSeq
    +        }
    +
    +        otherSideCandidates.foreach { otherSidePlan =>
    +          // Should not join two overlapping item sets.
    +          if (oneSidePlan.itemIds.intersect(otherSidePlan.itemIds).isEmpty) {
    +            val joinPlan = buildJoin(oneSidePlan, otherSidePlan, conf, conditions, topOutput)
    +            // Check if it's the first plan for the item set, or it's a better plan than
    +            // the existing one due to lower cost.
    +            val existingPlan = nextLevel.get(joinPlan.itemIds)
    +            if (existingPlan.isEmpty || joinPlan.cost.lessThan(existingPlan.get.cost)) {
    +              nextLevel.update(joinPlan.itemIds, joinPlan)
    +            }
    +          }
    +        }
    +      }
    +      k += 1
    +    }
    +    nextLevel.toMap
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(
    +      oneJoinPlan: JoinPlan,
    +      otherJoinPlan: JoinPlan,
    +      conf: CatalystConf,
    +      conditions: Set[Expression],
    +      topOutput: AttributeSet): JoinPlan = {
    +
    +    val onePlan = oneJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    // Now both onePlan and otherPlan become intermediate joins, so the cost of the
    +    // new join should also include their own cardinalities and sizes.
    +    val newCost = if (isCartesianProduct(onePlan) || isCartesianProduct(otherPlan)) {
    +      // We consider cartesian product very expensive, thus set a very large cost for it.
    +      // This enables to plan all the cartesian products at the end, because having a cartesian
    +      // product as an intermediate join will significantly increase a plan's cost, making it
    +      // impossible to be selected as the best plan for the items, unless there's no other choice.
    +      Cost(
    +        rows = BigInt(Long.MaxValue) * BigInt(Long.MaxValue),
    +        size = BigInt(Long.MaxValue) * BigInt(Long.MaxValue))
    +    } else {
    +      val onePlanStats = onePlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      Cost(
    +        rows = oneJoinPlan.cost.rows + onePlanStats.rowCount.get +
    +          otherJoinPlan.cost.rows + otherPlanStats.rowCount.get,
    +        size = oneJoinPlan.cost.size + onePlanStats.sizeInBytes +
    +          otherJoinPlan.cost.size + otherPlanStats.sizeInBytes)
    +    }
    +
    +    // Put the deeper side on the left, tend to build a left-deep tree.
    +    val (left, right) = if (oneJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +      (onePlan, otherPlan)
    +    } else {
    +      (otherPlan, onePlan)
    +    }
    +    val joinConds = conditions
    +      .filterNot(l => canEvaluate(l, onePlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(onePlan.outputSet ++ otherPlan.outputSet))
    +    // We use inner join whether join condition is empty or not. Since cross join is
    +    // equivalent to inner join without condition.
    +    val newJoin = Join(left, right, Inner, joinConds.reduceOption(And))
    +    val collectedJoinConds = joinConds ++ oneJoinPlan.joinConds ++ otherJoinPlan.joinConds
    +    val remainingConds = conditions -- collectedJoinConds
    +    val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +    val neededFromNewJoin = newJoin.outputSet.filter(neededAttr.contains)
    +    val newPlan =
    +      if ((newJoin.outputSet -- neededFromNewJoin).nonEmpty) {
    +        Project(neededFromNewJoin.toSeq, newJoin)
    +      } else {
    +        newJoin
    +      }
    +
    +    val itemIds = oneJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +    JoinPlan(itemIds, newPlan, collectedJoinConds, newCost)
    +  }
    +
    +  private def isCartesianProduct(plan: LogicalPlan): Boolean = plan match {
    +    case Join(_, _, _, None) => true
    +    case Project(_, Join(_, _, _, None)) => true
    +    case _ => false
    +  }
    +
    +  /** Map[set of item ids, join plan for these items] */
    +  type JoinPlanMap = Map[Set[Int], JoinPlan]
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param joinConds Join conditions included in the plan.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, joinConds: Set[Expression], cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, size: BigInt) {
    --- End diff --
    
    `rows` is confusing. Maybe `rowCount` or `numRows`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104174777
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    --- End diff --
    
    JoinEstimation estimates the output size and cardinality of a join node. Here the cost of a JoinPlan is the sum of costs of all intermediate joins.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104172818
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    --- End diff --
    
    Yes, I just want level i to represent plans for i items, it's more intuitive I think.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Merging to master. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104146673
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case _ =>
    +      Set()
    +  }
    +
    +  /**
    +   * Partial join order in a specific level.
    +   *
    +   * @param itemIds Set of item ids participating in this partial plan.
    +   * @param plan The plan tree with the lowest cost for these items found so far.
    +   * @param cost The cost of this plan is the sum of costs of all intermediate joins.
    +   */
    +  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, cost: Cost)
    +}
    +
    +/** This class defines the cost model. */
    +case class Cost(rows: BigInt, sizeInBytes: BigInt) {
    +  /**
    +   * An empirical value for the weights of cardinality (number of rows) in the cost formula:
    +   * cost = rows * weight + size * (1 - weight), usually cardinality is more important than size.
    +   */
    +  val weight = 0.7
    +
    +  def +(other: Cost): Cost = Cost(rows + other.rows, sizeInBytes + other.sizeInBytes)
    --- End diff --
    
    I am not sure if this is Ok according to the style guide :)...
    
    Same for `<`...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104152377
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    --- End diff --
    
    What is the rationale behind this cost calculation? Why aren't we using `JoinEstimation` here? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74199/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Found a very interesting paper for analyzing the existing DP algorithms for join reordering with a new solution in VLDB 2006. 
    [Analysis of Two Existing and One New Dynamic Programming Algorithm for the Generation of Optimal Bushy Join Trees without Cross Products](https://github.com/apache/spark/files/853071/10.1.1.153.5571.pdf)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104264964
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -288,6 +288,9 @@ case class Join(
         condition: Option[Expression])
       extends BinaryNode with PredicateHelper {
     
    +  /** Whether this join node is ordered before. Set true after it has been ordered. */
    +  var ordered: Boolean = false
    --- End diff --
    
    I don't think we need a flag, the join re-order optimizer rule only runs once.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106775177
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    --- End diff --
    
    By `items` I want to say that they are leaf plans. `subplan` sounds to include intermediate nodes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    cc @sameeragarwal @cloud-fan 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104867000
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    --- End diff --
    
    can't we just transform to the child join node and leave the project node on top?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104888504
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.optimizer
    +
    +import org.apache.spark.sql.catalyst.SimpleCatalystConf
    +import org.apache.spark.sql.catalyst.dsl.expressions._
    +import org.apache.spark.sql.catalyst.dsl.plans._
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
    +import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest}
    +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, LogicalPlan}
    +import org.apache.spark.sql.catalyst.rules.RuleExecutor
    +import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan}
    +import org.apache.spark.sql.catalyst.util._
    +
    +
    +class JoinReorderSuite extends PlanTest with StatsEstimationTestBase {
    --- End diff --
    
    NVM - missed the test case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    LGTM - pending jenkins. I left a few minor comments, but you can address those in a follow-up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104147505
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    --- End diff --
    
    Also make sure the predicates have been cannonicalized. That might improve your yield when reducing the plan.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104410276
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    --- End diff --
    
    yea good point, we should check every item's rowCount at the beginning. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104884534
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    --- End diff --
    
    Just for my understanding: you are doing this to capture an attribute only projection on top of a join? Projections with meaningful expressions will just return the original plan.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by nsyca <gi...@git.apache.org>.
Github user nsyca commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    You are right. The plans generated at n-join level comes from the join of the plans in (n-1)-join level as well as (n-2)-join level and so on. So it should be able to generate {A,B} join {C,D} plan.
    
    The fallback of this algorithm is the assumption that a lowest cost plan can only be generated among the lowest cost plans from its previous levels. An example is an {A,B} plan that could preserve the order of A that makes the join between {A,B} join {C,D} much less expensive may be discarded by this algorithm. In addition, because the decision to choose a sorted-merge join, which preserves the order of its input, versus other join methods is done in the `QueryPlanner` phase we do not have this information to make a good decision in the Optimizer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106797505
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    --- End diff --
    
    ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104265487
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    --- End diff --
    
    I think we should assume the project list expressions are all attribute


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104152016
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    --- End diff --
    
    This seems like a relatively expensive way to collect join conditions. Shouldn't we just add it to the `JoinPlan`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104136081
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    --- End diff --
    
    Use `k <= level - k` or something more like that?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104131186
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    +        // Now both curPlan and otherPlan become intermediate joins, so the cost of the
    +        // new join should also include their costs.
    +        val cost = curJoinPlan.cost + otherJoinPlan.cost +
    +          Cost(curPlanStats.rowCount.get, curPlanStats.sizeInBytes) +
    +          Cost(otherPlanStats.rowCount.get, otherPlanStats.sizeInBytes)
    +
    +        // Put the deeper side on the left, tend to build a left-deep tree.
    +        val (left, right) = if (curJoinPlan.itemIds.size >= otherJoinPlan.itemIds.size) {
    +          (curPlan, otherPlan)
    +        } else {
    +          (otherPlan, curPlan)
    +        }
    +        val newJoin = Join(left, right, Inner, joinCond.reduceOption(And))
    +        val remainingConds = conditions -- collectJoinConds(newJoin)
    +        val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
    +        val newPlan =
    +          if ((newJoin.outputSet -- newJoin.outputSet.filter(neededAttr.contains)).nonEmpty) {
    +            Project(newJoin.output.filter(neededAttr.contains), newJoin)
    +          } else {
    +            newJoin
    +          }
    +        val itemIds = curJoinPlan.itemIds.union(otherJoinPlan.itemIds)
    +        return Some(JoinPlan(itemIds, newPlan, cost))
    +      }
    +    }
    +    None
    +  }
    +
    +  private def collectJoinConds(plan: LogicalPlan): Set[Expression] = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val leftConditions = collectJoinConds(left)
    +      val rightConditions = collectJoinConds(right)
    +      cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++ leftConditions ++ rightConditions
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val leftConditions = collectJoinConds(left)
    --- End diff --
    
    Same comment: Just recurse into the join.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104370871
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    --- End diff --
    
    Yeah, just check attribute only projections.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73767/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104173778
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    --- End diff --
    
    After the `ReorderJoin` optimization rule, cross join are all at the end. Maybe I should use `Inner` join type instead of `InnerLike` while collecting joinable items, such that we don't need to worry about cross join during reordering?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104782460
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    --- End diff --
    
    @ron8hu I am not saying we should change the algorithm. I was just wondering how we selected it. I think it is a good idea to start with a proven algorithm, so I think`DPSize` is a good idea.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73772/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104864183
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    --- End diff --
    
    can we reuse `ExtractFiltersAndInnerJoins`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r106781914
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      val result = plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) =>
    +          reorder(p, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) =>
    +          reorder(j, j.outputSet)
    +      }
    +      // After reordering is finished, convert OrderedJoin back to Join
    +      result transform {
    +        case oj: OrderedJoin => oj.join
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    --- End diff --
    
    Sorry my words are not accurate. By `leaf plan` I mean basic reorder unit, it can be any node. I want to use `item` to mean it's a basic unit, while `subplan` can be any level reordered results.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    @nsyca This algorithm supports bushy tree, I also had a test case for it.
    About your second comment, yes you are right, we had a similar discussion [here](https://github.com/apache/spark/pull/17138#discussion_r104128753).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104287073
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    --- End diff --
    
    Sorry, I mean the other way around. When will estimation fail? Will we still converge if estimation fails?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    **[Test build #74207 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74207/testReport)** for PR 17138 at commit [`757e3b6`](https://github.com/apache/spark/commit/757e3b65083e860aba3d1161c914c547545a28f5).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, joinConds: Set[Expression], cost: Cost)`
      * `case class Cost(rows: BigInt, size: BigInt) `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104153912
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      val (leftPlans, leftConditions) = extractInnerJoins(left)
    +      val (rightPlans, rightConditions) = extractInnerJoins(right)
    +      (leftPlans ++ rightPlans, cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
    +        leftConditions ++ rightConditions)
    +    case _ =>
    +      (Seq(plan), Set())
    +  }
    +
    +  def setOrdered(plan: LogicalPlan): Unit = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
    +      j.ordered = true
    +      setOrdered(left)
    +      setOrdered(right)
    +    case _ =>
    +  }
    +}
    +
    +/**
    + * Reorder the joins using a dynamic programming algorithm:
    + * First we put all items (basic joined nodes) into level 1, then we build all two-way joins
    + * at level 2 from plans at level 1 (single items), then build all 3-way joins from plans
    + * at previous levels (two-way joins and single items), then 4-way joins ... etc, until we
    + * build all n-way joins and pick the best plan among them.
    + *
    + * When building m-way joins, we only keep the best plan (with the lowest cost) for the same set
    + * of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among
    + * plans (A J B) J C, (A J C) J B and (B J C) J A.
    + *
    + * Thus the plans maintained for each level when reordering four items A, B, C, D are as follows:
    + * level 1: p({A}), p({B}), p({C}), p({D})
    + * level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
    + * level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
    + * level 4: p({A, B, C, D})
    + * where p({A, B, C, D}) is the final output plan.
    + *
    + * For cost evaluation, since physical costs for operators are not available currently, we use
    + * cardinalities and sizes to compute costs.
    + */
    +case class JoinReorderDP(
    +    conf: CatalystConf,
    +    items: Seq[LogicalPlan],
    +    conditions: Set[Expression],
    +    topOutput: AttributeSet) extends PredicateHelper{
    +
    +  /** Level i maintains all found plans for sets of i joinable items. */
    +  val foundPlans = new Array[mutable.Map[Set[Int], JoinPlan]](items.length + 1)
    +  for (i <- 1 to items.length) foundPlans(i) = mutable.Map.empty
    +
    +  def search(): Option[LogicalPlan] = {
    +    // Start from the first level: each plan is a single item with zero cost.
    +    val itemIndex = items.zipWithIndex
    +    foundPlans(1) ++=
    +      itemIndex.map { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Cost(0, 0)) }
    +
    +    for (lev <- 2 to items.length) {
    +      searchForLevel(lev)
    +    }
    +
    +    val plansLastLevel = foundPlans(items.length)
    +    if (plansLastLevel.isEmpty) {
    +      // Failed to find a plan, fall back to the original plan
    +      None
    +    } else {
    +      // There must be only one plan at the last level, which contains all items.
    +      assert(plansLastLevel.size == 1 && plansLastLevel.head._1.size == items.length)
    +      Some(plansLastLevel.head._2.plan)
    +    }
    +  }
    +
    +  /** Find all possible plans in one level, based on previous levels. */
    +  private def searchForLevel(level: Int): Unit = {
    +    val foundPlansCurLevel = foundPlans(level)
    +    var k = 1
    +    var continue = true
    +    while (continue) {
    +      val otherLevel = level - k
    +      if (k > otherLevel) {
    +        // We can break from here, because when building a join from A and B, both A J B and B J A
    +        // are handled.
    +        continue = false
    +      } else {
    +        val joinPlansLevelK = foundPlans(k).values.toSeq
    +        for (i <- joinPlansLevelK.indices) {
    +          val curJoinPlan = joinPlansLevelK(i)
    +
    +          val joinPlansOtherLevel = if (k == otherLevel) {
    +            // Both sides of a join are at the same level, no need to repeat for previous ones.
    +            joinPlansLevelK.drop(i)
    +          } else {
    +            foundPlans(otherLevel).values.toSeq
    +          }
    +
    +          joinPlansOtherLevel.foreach { otherJoinPlan =>
    +            // Should not join two overlapping item sets.
    +            if (curJoinPlan.itemIds.intersect(otherJoinPlan.itemIds).isEmpty) {
    +              val joinPlan = buildJoin(curJoinPlan, otherJoinPlan)
    +              if (joinPlan.nonEmpty) {
    +                // Check if it's the first plan for the item set, or it's a better plan than
    +                // the existing one due to lower cost.
    +                val existingPlan = foundPlansCurLevel.get(joinPlan.get.itemIds)
    +                if (existingPlan.isEmpty || joinPlan.get.cost < existingPlan.get.cost) {
    +                  foundPlansCurLevel.update(joinPlan.get.itemIds, joinPlan.get)
    +                }
    +              }
    +            }
    +          }
    +        }
    +
    +        k += 1
    +      }
    +    }
    +  }
    +
    +  /** Build a new join node. */
    +  private def buildJoin(curJoinPlan: JoinPlan, otherJoinPlan: JoinPlan): Option[JoinPlan] = {
    +    // Check if these two nodes are inner joinable. We consider cartesian product very
    +    // costly, thus exclude such plans. This also helps us to reduce the search space.
    +    val curPlan = curJoinPlan.plan
    +    val otherPlan = otherJoinPlan.plan
    +    val joinCond = conditions
    +      .filterNot(l => canEvaluate(l, curPlan))
    +      .filterNot(r => canEvaluate(r, otherPlan))
    +      .filter(e => e.references.subsetOf(curPlan.outputSet ++ otherPlan.outputSet))
    +
    +    if (joinCond.nonEmpty) {
    +      val curPlanStats = curPlan.stats(conf)
    +      val otherPlanStats = otherPlan.stats(conf)
    +      if (curPlanStats.rowCount.nonEmpty && otherPlanStats.rowCount.nonEmpty) {
    --- End diff --
    
    When does this happen?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104865304
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    --- End diff --
    
    I need to know the final output attributes to do column pruning when building new join nodes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    **[Test build #74199 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74199/testReport)** for PR 17138 at commit [`757e3b6`](https://github.com/apache/spark/commit/757e3b65083e860aba3d1161c914c547545a28f5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, joinConds: Set[Expression], cost: Cost)`
      * `case class Cost(rows: BigInt, size: BigInt) `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    **[Test build #74199 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74199/testReport)** for PR 17138 at commit [`757e3b6`](https://github.com/apache/spark/commit/757e3b65083e860aba3d1161c914c547545a28f5).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104885896
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -118,6 +118,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf)
           SimplifyCreateMapOps) ::
         Batch("Check Cartesian Products", Once,
           CheckCartesianProducts(conf)) ::
    +    Batch("Join Reorder", Once,
    +      CostBasedJoinReorder(conf)) ::
    --- End diff --
    
    CostBasedJoinReorder can create two consecutive projections. The top projection will have some expressions, and the bottom projection will only have attributes (it is created by the join reorder rule). We can get rid of these stacked projections by calling `CollapseProject` rule after the reordering.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    @nsyca Yes you're right. There's still much room of optimization. We will improve Spark's optimizer gradually :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17138: [SPARK-17080] [SQL] join reorder

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17138#discussion_r104130797
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala ---
    @@ -0,0 +1,274 @@
    +/*
    + * 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.optimizer
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.catalyst.CatalystConf
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, PredicateHelper}
    +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
    +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +
    +/**
    + * Cost-based join reorder.
    + * We may have several join reorder algorithms in the future. This class is the entry of these
    + * algorithms, and chooses which one to use.
    + */
    +case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.cboEnabled || !conf.joinReorderEnabled) {
    +      plan
    +    } else {
    +      plan transform {
    +        case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if !j.ordered =>
    +          reorder(j, p.outputSet)
    +        case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
    +          reorder(j, j.outputSet)
    +      }
    +    }
    +  }
    +
    +  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
    +    val (items, conditions) = extractInnerJoins(plan)
    +    val result =
    +      if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && conditions.nonEmpty) {
    +        JoinReorderDP(conf, items, conditions, output).search().getOrElse(plan)
    +      } else {
    +        plan
    +      }
    +    // Set all inside joins ordered.
    +    setOrdered(result)
    +    result
    +  }
    +
    +  /**
    +   * Extract inner joinable items and join conditions.
    +   * This method works for bushy trees and left/right deep trees.
    +   */
    +  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = plan match {
    +    case j @ Join(left, right, _: InnerLike, cond) =>
    --- End diff --
    
    Can we also add LeftSemi joins here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74207/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17138: [SPARK-17080] [SQL] join reorder

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17138
  
    **[Test build #73772 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73772/testReport)** for PR 17138 at commit [`cb35885`](https://github.com/apache/spark/commit/cb35885873c81638ea116c874576787c030335cb).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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