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 @@ -116,7 +116,7 @@ class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: A
class ShuffledRowRDD(
var dependency: ShuffleDependency[Int, InternalRow, InternalRow],
metrics: Map[String, SQLMetric],
specifiedPartitionIndices: Option[Array[(Int, Int)]] = None)
specifiedPartitionStartIndices: Option[Array[Int]] = None)
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

just revert changes made to this file in #26434

extends RDD[InternalRow](dependency.rdd.context, Nil) {

if (SQLConf.get.fetchShuffleBlocksInBatchEnabled) {
Expand All @@ -126,8 +126,8 @@ class ShuffledRowRDD(

private[this] val numPreShufflePartitions = dependency.partitioner.numPartitions

private[this] val partitionStartIndices: Array[Int] = specifiedPartitionIndices match {
case Some(indices) => indices.map(_._1)
private[this] val partitionStartIndices: Array[Int] = specifiedPartitionStartIndices match {
case Some(indices) => indices
case None =>
// When specifiedPartitionStartIndices is not defined, every post-shuffle partition
// corresponds to a pre-shuffle partition.
Expand All @@ -142,15 +142,16 @@ class ShuffledRowRDD(
override val partitioner: Option[Partitioner] = Some(part)

override def getPartitions: Array[Partition] = {
specifiedPartitionIndices match {
case Some(indices) =>
Array.tabulate[Partition](indices.length) { i =>
new ShuffledRowRDDPartition(i, indices(i)._1, indices(i)._2)
}
case None =>
Array.tabulate[Partition](numPreShufflePartitions) { i =>
new ShuffledRowRDDPartition(i, i, i + 1)
assert(partitionStartIndices.length == part.numPartitions)
Array.tabulate[Partition](partitionStartIndices.length) { i =>
val startIndex = partitionStartIndices(i)
val endIndex =
if (i < partitionStartIndices.length - 1) {
partitionStartIndices(i + 1)
} else {
numPreShufflePartitions
}
new ShuffledRowRDDPartition(i, startIndex, endIndex)
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
/*
* 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.adaptive

import org.apache.spark.{Dependency, MapOutputTrackerMaster, Partition, ShuffleDependency, SparkEnv, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter}

sealed trait ShufflePartitionSpec

// A partition that reads data of one reducer.
case class SinglePartitionSpec(reducerIndex: Int) extends ShufflePartitionSpec

// A partition that reads data of multiple reducers, from `startReducerIndex` (inclusive) to
// `endReducerIndex` (exclusive).
case class CoalescedPartitionSpec(
startReducerIndex: Int, endReducerIndex: Int) extends ShufflePartitionSpec

// A partition that reads partial data of one reducer, from `startMapIndex` (inclusive) to
// `endMapIndex` (exclusive).
case class PartialPartitionSpec(
reducerIndex: Int, startMapIndex: Int, endMapIndex: Int) extends ShufflePartitionSpec

private final case class CustomShufflePartition(
index: Int, spec: ShufflePartitionSpec) extends Partition

// TODO: merge this with `ShuffledRowRDD`, and replace `LocalShuffledRowRDD` with this RDD.
class CustomShuffledRowRDD(
var dependency: ShuffleDependency[Int, InternalRow, InternalRow],
metrics: Map[String, SQLMetric],
partitionSpecs: Array[ShufflePartitionSpec])
extends RDD[InternalRow](dependency.rdd.context, Nil) {

override def getDependencies: Seq[Dependency[_]] = List(dependency)

override def clearDependencies() {
super.clearDependencies()
dependency = null
}

override def getPartitions: Array[Partition] = {
Array.tabulate[Partition](partitionSpecs.length) { i =>
CustomShufflePartition(i, partitionSpecs(i))
}
}

override def getPreferredLocations(partition: Partition): Seq[String] = {
val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]
partition.asInstanceOf[CustomShufflePartition].spec match {
case SinglePartitionSpec(reducerIndex) =>
tracker.getPreferredLocationsForShuffle(dependency, reducerIndex)

case CoalescedPartitionSpec(startReducerIndex, endReducerIndex) =>
startReducerIndex.until(endReducerIndex).flatMap { reducerIndex =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

More for a follow-up: Is there a way we can order the preferred locations by size? Note that this is already a net improvement over the ShuffledRowRDD where we would use the incorrect reducer

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds like a good idea. We may need tracker.getPreferredLocationsForShuffle to return size as well so it involves more changes. Let's leave it for followup.

tracker.getPreferredLocationsForShuffle(dependency, reducerIndex)
}

case PartialPartitionSpec(_, startMapIndex, endMapIndex) =>
tracker.getMapLocation(dependency, startMapIndex, endMapIndex)
}
}

override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = {
val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics()
// `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator,
// as well as the `tempMetrics` for basic shuffle metrics.
val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics)
val reader = split.asInstanceOf[CustomShufflePartition].spec match {
case SinglePartitionSpec(reducerIndex) =>
SparkEnv.get.shuffleManager.getReader(
dependency.shuffleHandle,
reducerIndex,
reducerIndex + 1,
context,
sqlMetricsReporter)

case CoalescedPartitionSpec(startReducerIndex, endReducerIndex) =>
SparkEnv.get.shuffleManager.getReader(
dependency.shuffleHandle,
startReducerIndex,
endReducerIndex,
context,
sqlMetricsReporter)

case PartialPartitionSpec(reducerIndex, startMapIndex, endMapIndex) =>
SparkEnv.get.shuffleManager.getReaderForRange(
dependency.shuffleHandle,
startMapIndex,
endMapIndex,
reducerIndex,
reducerIndex + 1,
context,
sqlMetricsReporter)
}
reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] {
plan match {
case c @ CoalescedShuffleReaderExec(s: ShuffleQueryStageExec, _) =>
LocalShuffleReaderExec(
s, getPartitionStartIndices(s, Some(c.partitionIndices.length)))
s, getPartitionStartIndices(s, Some(c.partitionStartIndices.length)))
case s: ShuffleQueryStageExec =>
LocalShuffleReaderExec(s, getPartitionStartIndices(s, None))
}
Expand Down
Loading