Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] {
val numExchangeAfter = numExchanges(EnsureRequirements(conf).apply(optimizedPlan))

if (numExchangeAfter > numExchangeBefore) {
logWarning("OptimizeLocalShuffleReader rule is not applied due" +
logDebug("OptimizeLocalShuffleReader rule is not applied due" +
" to additional shuffles will be introduced.")
plan
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.adaptive

import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan}
import org.apache.spark.sql.execution.adaptive.rule.CoalescedShuffleReaderExec
import org.apache.spark.sql.execution.exchange.Exchange
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildRight, SortMergeJoinExec}
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -78,7 +77,7 @@ class AdaptiveQueryExecSuite
}

private def checkNumLocalShuffleReaders(plan: SparkPlan, expected: Int): Unit = {
val localReaders = plan.collect {
val localReaders = collect(plan) {
case reader: LocalShuffleReaderExec => reader
}
assert(localReaders.length === expected)
Expand Down Expand Up @@ -164,7 +163,7 @@ class AdaptiveQueryExecSuite
assert(smj.size == 3)
val bhj = findTopLevelBroadcastHashJoin(adaptivePlan)
assert(bhj.size == 3)

// additional shuffle exchange introduced, only one shuffle reader to local shuffle reader.
checkNumLocalShuffleReaders(adaptivePlan, 1)
}
}
Expand All @@ -189,8 +188,8 @@ class AdaptiveQueryExecSuite
assert(smj.size == 3)
val bhj = findTopLevelBroadcastHashJoin(adaptivePlan)
assert(bhj.size == 3)

checkNumLocalShuffleReaders(adaptivePlan, 0)
// additional shuffle exchange introduced, only one shuffle reader to local shuffle reader.
checkNumLocalShuffleReaders(adaptivePlan, 1)
}
}

Expand All @@ -214,7 +213,8 @@ class AdaptiveQueryExecSuite
assert(smj.size == 3)
val bhj = findTopLevelBroadcastHashJoin(adaptivePlan)
assert(bhj.size == 3)
checkNumLocalShuffleReaders(adaptivePlan, 0)
// additional shuffle exchange introduced, only one shuffle reader to local shuffle reader.
checkNumLocalShuffleReaders(adaptivePlan, 1)
}
}

Expand All @@ -229,6 +229,8 @@ class AdaptiveQueryExecSuite
assert(smj.size == 3)
val bhj = findTopLevelBroadcastHashJoin(adaptivePlan)
assert(bhj.size == 2)
checkNumLocalShuffleReaders(adaptivePlan, 2)
// Even with local shuffle reader, the query statge reuse can also work.
val ex = findReusedExchange(adaptivePlan)
assert(ex.size == 1)
}
Expand All @@ -245,6 +247,8 @@ class AdaptiveQueryExecSuite
assert(smj.size == 1)
val bhj = findTopLevelBroadcastHashJoin(adaptivePlan)
assert(bhj.size == 1)
checkNumLocalShuffleReaders(adaptivePlan, 1)
// Even with local shuffle reader, the query statge reuse can also work.
val ex = findReusedExchange(adaptivePlan)
assert(ex.size == 1)
}
Expand All @@ -263,6 +267,8 @@ class AdaptiveQueryExecSuite
assert(smj.size == 1)
val bhj = findTopLevelBroadcastHashJoin(adaptivePlan)
assert(bhj.size == 1)
checkNumLocalShuffleReaders(adaptivePlan, 1)
// Even with local shuffle reader, the query statge reuse can also work.
val ex = findReusedExchange(adaptivePlan)
assert(ex.nonEmpty)
val sub = findReusedSubquery(adaptivePlan)
Expand All @@ -282,6 +288,8 @@ class AdaptiveQueryExecSuite
assert(smj.size == 1)
val bhj = findTopLevelBroadcastHashJoin(adaptivePlan)
assert(bhj.size == 1)
checkNumLocalShuffleReaders(adaptivePlan, 1)
// Even with local shuffle reader, the query statge reuse can also work.
val ex = findReusedExchange(adaptivePlan)
assert(ex.isEmpty)
val sub = findReusedSubquery(adaptivePlan)
Expand All @@ -304,6 +312,8 @@ class AdaptiveQueryExecSuite
assert(smj.size == 1)
val bhj = findTopLevelBroadcastHashJoin(adaptivePlan)
assert(bhj.size == 1)
checkNumLocalShuffleReaders(adaptivePlan, 1)
// Even with local shuffle reader, the query statge reuse can also work.
val ex = findReusedExchange(adaptivePlan)
assert(ex.nonEmpty)
assert(ex.head.plan.isInstanceOf[BroadcastQueryStageExec])
Expand Down