[SPARK-13598] [SQL] remove LeftSemiJoinBNL

## What changes were proposed in this pull request?

Broadcast left semi join without joining keys is already supported in BroadcastNestedLoopJoin, it has the same implementation as LeftSemiJoinBNL, we should remove that.

## How was this patch tested?

Updated unit tests.

Author: Davies Liu <davies@databricks.com>

Closes #11448 from davies/remove_bnl.
This commit is contained in:
Davies Liu 2016-03-01 17:27:57 -08:00 committed by Davies Liu
parent b0ee7d4373
commit a640c5b4fb
4 changed files with 2 additions and 95 deletions

View file

@ -71,9 +71,6 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) =>
joins.LeftSemiJoinHash(
leftKeys, rightKeys, planLater(left), planLater(right), condition) :: Nil
// no predicate can be evaluated by matching hash keys
case logical.Join(left, right, LeftSemi, condition) =>
joins.LeftSemiJoinBNL(planLater(left), planLater(right), condition) :: Nil
case _ => Nil
}
}

View file

@ -1,80 +0,0 @@
/*
* 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.execution.joins
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.metric.SQLMetrics
/**
* Using BroadcastNestedLoopJoin to calculate left semi join result when there's no join keys
* for hash join.
*/
case class LeftSemiJoinBNL(
streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression]) extends BinaryNode {
override private[sql] lazy val metrics = Map(
"numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
override def outputPartitioning: Partitioning = streamed.outputPartitioning
override def output: Seq[Attribute] = left.output
/** The Streamed Relation */
override def left: SparkPlan = streamed
/** The Broadcast relation */
override def right: SparkPlan = broadcast
override def requiredChildDistribution: Seq[Distribution] = {
UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil
}
@transient private lazy val boundCondition =
newPredicate(condition.getOrElse(Literal(true)), left.output ++ right.output)
protected override def doExecute(): RDD[InternalRow] = {
val numOutputRows = longMetric("numOutputRows")
val broadcastedRelation = broadcast.executeBroadcast[Array[InternalRow]]()
streamed.execute().mapPartitions { streamedIter =>
val joinedRow = new JoinedRow
val relation = broadcastedRelation.value
streamedIter.filter(streamedRow => {
var i = 0
var matched = false
while (i < relation.length && !matched) {
if (boundCondition(joinedRow(streamedRow, relation(i)))) {
matched = true
}
i += 1
}
if (matched) {
numOutputRows += 1
}
matched
})
}
}
}

View file

@ -47,7 +47,6 @@ class JoinSuite extends QueryTest with SharedSQLContext {
val operators = physical.collect {
case j: LeftSemiJoinHash => j
case j: BroadcastHashJoin => j
case j: LeftSemiJoinBNL => j
case j: CartesianProduct => j
case j: BroadcastNestedLoopJoin => j
case j: BroadcastLeftSemiJoinHash => j
@ -67,7 +66,7 @@ class JoinSuite extends QueryTest with SharedSQLContext {
withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0") {
Seq(
("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]),
("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[LeftSemiJoinBNL]),
("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[BroadcastNestedLoopJoin]),
("SELECT * FROM testData JOIN testData2", classOf[CartesianProduct]),
("SELECT * FROM testData JOIN testData2 WHERE key = 2", classOf[CartesianProduct]),
("SELECT * FROM testData LEFT JOIN testData2", classOf[BroadcastNestedLoopJoin]),
@ -465,7 +464,7 @@ class JoinSuite extends QueryTest with SharedSQLContext {
("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a",
classOf[LeftSemiJoinHash]),
("SELECT * FROM testData LEFT SEMI JOIN testData2",
classOf[LeftSemiJoinBNL]),
classOf[BroadcastNestedLoopJoin]),
("SELECT * FROM testData JOIN testData2",
classOf[BroadcastNestedLoopJoin]),
("SELECT * FROM testData JOIN testData2 WHERE key = 2",

View file

@ -95,15 +95,6 @@ class SemiJoinSuite extends SparkPlanTest with SharedSQLContext {
}
}
test(s"$testName using LeftSemiJoinBNL") {
withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) =>
LeftSemiJoinBNL(left, right, Some(condition)),
expectedAnswer.map(Row.fromTuple),
sortAnswers = true)
}
}
test(s"$testName using BroadcastNestedLoopJoin build left") {
withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) =>