Skip to content

Commit

Permalink
unrevert the aggregation related code
Browse files Browse the repository at this point in the history
  • Loading branch information
JkSelf committed Dec 11, 2018
1 parent 03cfe2b commit a46d18e
Show file tree
Hide file tree
Showing 4 changed files with 6 additions and 126 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Dist
import org.apache.spark.sql.execution.{BinaryExecNode, CodegenSupport, SparkPlan}
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.types.{BooleanType, LongType}
import org.apache.spark.util.TaskCompletionListener

/**
* Performs an inner hash join of two child relations. When the output RDD of this operator is
Expand All @@ -48,8 +47,7 @@ case class BroadcastHashJoinExec(
extends BinaryExecNode with HashJoin with CodegenSupport {

override lazy val metrics = Map(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
"avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe"))
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"))

override def requiredChildDistribution: Seq[Distribution] = {
val mode = HashedRelationBroadcastMode(buildKeys)
Expand All @@ -63,13 +61,12 @@ case class BroadcastHashJoinExec(

protected override def doExecute(): RDD[InternalRow] = {
val numOutputRows = longMetric("numOutputRows")
val avgHashProbe = longMetric("avgHashProbe")

val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]()
streamedPlan.execute().mapPartitions { streamedIter =>
val hashed = broadcastRelation.value.asReadOnlyCopy()
TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize)
join(streamedIter, hashed, numOutputRows, avgHashProbe)
join(streamedIter, hashed, numOutputRows)
}
}

Expand Down Expand Up @@ -111,23 +108,6 @@ case class BroadcastHashJoinExec(
}
}

/**
* Returns the codes used to add a task completion listener to update avg hash probe
* at the end of the task.
*/
private def genTaskListener(avgHashProbe: String, relationTerm: String): String = {
val listenerClass = classOf[TaskCompletionListener].getName
val taskContextClass = classOf[TaskContext].getName
s"""
| $taskContextClass$$.MODULE$$.get().addTaskCompletionListener(new $listenerClass() {
| @Override
| public void onTaskCompletion($taskContextClass context) {
| $avgHashProbe.set($relationTerm.getAverageProbesPerLookup());
| }
| });
""".stripMargin
}

/**
* Returns a tuple of Broadcast of HashedRelation and the variable name for it.
*/
Expand All @@ -137,15 +117,11 @@ case class BroadcastHashJoinExec(
val broadcast = ctx.addReferenceObj("broadcast", broadcastRelation)
val clsName = broadcastRelation.value.getClass.getName

// At the end of the task, we update the avg hash probe.
val avgHashProbe = metricTerm(ctx, "avgHashProbe")

// Inline mutable state since not many join operations in a task
val relationTerm = ctx.addMutableState(clsName, "relation",
v => s"""
| $v = (($clsName) $broadcast.value()).asReadOnlyCopy();
| incPeakExecutionMemory($v.estimatedSize());
| ${genTaskListener(avgHashProbe, v)}
""".stripMargin, forceInline = true)
(broadcastRelation, relationTerm)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark.sql.execution.joins

import org.apache.spark.TaskContext
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
Expand Down Expand Up @@ -194,8 +193,7 @@ trait HashJoin {
protected def join(
streamedIter: Iterator[InternalRow],
hashed: HashedRelation,
numOutputRows: SQLMetric,
avgHashProbe: SQLMetric): Iterator[InternalRow] = {
numOutputRows: SQLMetric): Iterator[InternalRow] = {

val joinedIter = joinType match {
case _: InnerLike =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,8 +42,7 @@ case class ShuffledHashJoinExec(
override lazy val metrics = Map(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
"buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"),
"buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map"),
"avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe"))
"buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map"))

override def requiredChildDistribution: Seq[Distribution] =
HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil
Expand All @@ -63,10 +62,9 @@ case class ShuffledHashJoinExec(

protected override def doExecute(): RDD[InternalRow] = {
val numOutputRows = longMetric("numOutputRows")
val avgHashProbe = longMetric("avgHashProbe")
streamedPlan.execute().zipPartitions(buildPlan.execute()) { (streamIter, buildIter) =>
val hashed = buildHashedRelation(buildIter)
join(streamIter, hashed, numOutputRows, avgHashProbe)
join(streamIter, hashed, numOutputRows)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -261,50 +261,6 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared
)
}

test("BroadcastHashJoin metrics: track avg probe") {
// The executed plan looks like:
// Project [a#210, b#211, b#221]
// +- BroadcastHashJoin [a#210], [a#220], Inner, BuildRight
// :- Project [_1#207 AS a#210, _2#208 AS b#211]
// : +- Filter isnotnull(_1#207)
// : +- LocalTableScan [_1#207, _2#208]
// +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, binary, true]))
// +- Project [_1#217 AS a#220, _2#218 AS b#221]
// +- Filter isnotnull(_1#217)
// +- LocalTableScan [_1#217, _2#218]
//
// Assume the execution plan with node id is
// WholeStageCodegen disabled:
// Project(nodeId = 0)
// BroadcastHashJoin(nodeId = 1)
// ...(ignored)
//
// WholeStageCodegen enabled:
// WholeStageCodegen(nodeId = 0)
// Project(nodeId = 1)
// BroadcastHashJoin(nodeId = 2)
// Project(nodeId = 3)
// Filter(nodeId = 4)
// ...(ignored)
Seq(true, false).foreach { enableWholeStage =>
val df1 = generateRandomBytesDF()
val df2 = generateRandomBytesDF()
val df = df1.join(broadcast(df2), "a")
val nodeIds = if (enableWholeStage) {
Set(2L)
} else {
Set(1L)
}
val metrics = getSparkPlanMetrics(df, 2, nodeIds, enableWholeStage).get
nodeIds.foreach { nodeId =>
val probes = metrics(nodeId)._2("avg hash probe (min, med, max)")
probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe =>
assert(probe.toDouble > 1.0)
}
}
}
}

test("ShuffledHashJoin metrics") {
withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "40",
"spark.sql.shuffle.partitions" -> "2",
Expand All @@ -323,8 +279,7 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared
val df = df1.join(df2, "key")
testSparkPlanMetrics(df, 1, Map(
1L -> (("ShuffledHashJoin", Map(
"number of output rows" -> 2L,
"avg hash probe (min, med, max)" -> "\n(1, 1, 1)"))),
"number of output rows" -> 2L))),
2L -> (("Exchange", Map(
"shuffle records written" -> 2L,
"records read" -> 2L))),
Expand All @@ -335,53 +290,6 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared
}
}

test("ShuffledHashJoin metrics: track avg probe") {
// The executed plan looks like:
// Project [a#308, b#309, b#319]
// +- ShuffledHashJoin [a#308], [a#318], Inner, BuildRight
// :- Exchange hashpartitioning(a#308, 2)
// : +- Project [_1#305 AS a#308, _2#306 AS b#309]
// : +- Filter isnotnull(_1#305)
// : +- LocalTableScan [_1#305, _2#306]
// +- Exchange hashpartitioning(a#318, 2)
// +- Project [_1#315 AS a#318, _2#316 AS b#319]
// +- Filter isnotnull(_1#315)
// +- LocalTableScan [_1#315, _2#316]
//
// Assume the execution plan with node id is
// WholeStageCodegen disabled:
// Project(nodeId = 0)
// ShuffledHashJoin(nodeId = 1)
// ...(ignored)
//
// WholeStageCodegen enabled:
// WholeStageCodegen(nodeId = 0)
// Project(nodeId = 1)
// ShuffledHashJoin(nodeId = 2)
// ...(ignored)
withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "5000000",
"spark.sql.shuffle.partitions" -> "2",
"spark.sql.join.preferSortMergeJoin" -> "false") {
Seq(true, false).foreach { enableWholeStage =>
val df1 = generateRandomBytesDF(65535 * 5)
val df2 = generateRandomBytesDF(65535)
val df = df1.join(df2, "a")
val nodeIds = if (enableWholeStage) {
Set(2L)
} else {
Set(1L)
}
val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get
nodeIds.foreach { nodeId =>
val probes = metrics(nodeId)._2("avg hash probe (min, med, max)")
probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe =>
assert(probe.toDouble > 1.0)
}
}
}
}
}

test("BroadcastHashJoin(outer) metrics") {
val df1 = Seq((1, "a"), (1, "b"), (4, "c")).toDF("key", "value")
val df2 = Seq((1, "a"), (1, "b"), (2, "c"), (3, "d")).toDF("key2", "value")
Expand Down

0 comments on commit a46d18e

Please sign in to comment.