aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src
diff options
context:
space:
mode:
authorDavies Liu <davies@databricks.com>2016-03-01 17:27:57 -0800
committerDavies Liu <davies.liu@gmail.com>2016-03-01 17:27:57 -0800
commita640c5b4fbd653919a5897a7b11f16328f2094eb (patch)
tree3b05d029e97a0c4e46d3f0838d02a6abd3bacbaf /sql/core/src
parentb0ee7d43730469ad61fdf6b7b75cc1b1efb62c31 (diff)
downloadspark-a640c5b4fbd653919a5897a7b11f16328f2094eb.tar.gz
spark-a640c5b4fbd653919a5897a7b11f16328f2094eb.tar.bz2
spark-a640c5b4fbd653919a5897a7b11f16328f2094eb.zip
[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.
Diffstat (limited to 'sql/core/src')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala3
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala80
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala5
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala9
4 files changed, 2 insertions, 95 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index dd8c96d5fa..0255103b63 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -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
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala
deleted file mode 100644
index df6dac8818..0000000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala
+++ /dev/null
@@ -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
- })
- }
- }
-}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala
index 3dab848e7b..5b98c11ef2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala
@@ -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",
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala
index 355f916a97..bc341db557 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala
@@ -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) =>