|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.execution.adaptive |
| 19 | + |
| 20 | +import org.apache.spark.{Dependency, MapOutputTrackerMaster, Partition, ShuffleDependency, SparkEnv, TaskContext} |
| 21 | +import org.apache.spark.rdd.RDD |
| 22 | +import org.apache.spark.sql.catalyst.InternalRow |
| 23 | +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} |
| 24 | + |
| 25 | +sealed trait ShufflePartitionSpec |
| 26 | + |
| 27 | +// A partition that reads data of one reducer. |
| 28 | +case class SinglePartitionSpec(reducerIndex: Int) extends ShufflePartitionSpec |
| 29 | + |
| 30 | +// A partition that reads data of multiple reducers, from `startReducerIndex` (inclusive) to |
| 31 | +// `endReducerIndex` (exclusive). |
| 32 | +case class CoalescedPartitionSpec( |
| 33 | + startReducerIndex: Int, endReducerIndex: Int) extends ShufflePartitionSpec |
| 34 | + |
| 35 | +// A partition that reads partial data of one reducer, from `startMapIndex` (inclusive) to |
| 36 | +// `endMapIndex` (exclusive). |
| 37 | +case class PartialPartitionSpec( |
| 38 | + reducerIndex: Int, startMapIndex: Int, endMapIndex: Int) extends ShufflePartitionSpec |
| 39 | + |
| 40 | +private final case class CustomShufflePartition( |
| 41 | + index: Int, spec: ShufflePartitionSpec) extends Partition |
| 42 | + |
| 43 | +// TODO: merge this with `ShuffledRowRDD`, and replace `LocalShuffledRowRDD` with this RDD. |
| 44 | +class CustomShuffledRowRDD( |
| 45 | + var dependency: ShuffleDependency[Int, InternalRow, InternalRow], |
| 46 | + metrics: Map[String, SQLMetric], |
| 47 | + partitionSpecs: Array[ShufflePartitionSpec]) |
| 48 | + extends RDD[InternalRow](dependency.rdd.context, Nil) { |
| 49 | + |
| 50 | + override def getDependencies: Seq[Dependency[_]] = List(dependency) |
| 51 | + |
| 52 | + override def clearDependencies() { |
| 53 | + super.clearDependencies() |
| 54 | + dependency = null |
| 55 | + } |
| 56 | + |
| 57 | + override def getPartitions: Array[Partition] = { |
| 58 | + Array.tabulate[Partition](partitionSpecs.length) { i => |
| 59 | + CustomShufflePartition(i, partitionSpecs(i)) |
| 60 | + } |
| 61 | + } |
| 62 | + |
| 63 | + override def getPreferredLocations(partition: Partition): Seq[String] = { |
| 64 | + val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] |
| 65 | + partition.asInstanceOf[CustomShufflePartition].spec match { |
| 66 | + case SinglePartitionSpec(reducerIndex) => |
| 67 | + tracker.getPreferredLocationsForShuffle(dependency, reducerIndex) |
| 68 | + |
| 69 | + case CoalescedPartitionSpec(startReducerIndex, endReducerIndex) => |
| 70 | + startReducerIndex.until(endReducerIndex).flatMap { reducerIndex => |
| 71 | + tracker.getPreferredLocationsForShuffle(dependency, reducerIndex) |
| 72 | + } |
| 73 | + |
| 74 | + case PartialPartitionSpec(_, startMapIndex, endMapIndex) => |
| 75 | + tracker.getMapLocation(dependency, startMapIndex, endMapIndex) |
| 76 | + } |
| 77 | + } |
| 78 | + |
| 79 | + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { |
| 80 | + val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() |
| 81 | + // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, |
| 82 | + // as well as the `tempMetrics` for basic shuffle metrics. |
| 83 | + val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) |
| 84 | + val reader = split.asInstanceOf[CustomShufflePartition].spec match { |
| 85 | + case SinglePartitionSpec(reducerIndex) => |
| 86 | + SparkEnv.get.shuffleManager.getReader( |
| 87 | + dependency.shuffleHandle, |
| 88 | + reducerIndex, |
| 89 | + reducerIndex + 1, |
| 90 | + context, |
| 91 | + sqlMetricsReporter) |
| 92 | + |
| 93 | + case CoalescedPartitionSpec(startReducerIndex, endReducerIndex) => |
| 94 | + SparkEnv.get.shuffleManager.getReader( |
| 95 | + dependency.shuffleHandle, |
| 96 | + startReducerIndex, |
| 97 | + endReducerIndex, |
| 98 | + context, |
| 99 | + sqlMetricsReporter) |
| 100 | + |
| 101 | + case PartialPartitionSpec(reducerIndex, startMapIndex, endMapIndex) => |
| 102 | + SparkEnv.get.shuffleManager.getReaderForRange( |
| 103 | + dependency.shuffleHandle, |
| 104 | + startMapIndex, |
| 105 | + endMapIndex, |
| 106 | + reducerIndex, |
| 107 | + reducerIndex + 1, |
| 108 | + context, |
| 109 | + sqlMetricsReporter) |
| 110 | + } |
| 111 | + reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) |
| 112 | + } |
| 113 | +} |
0 commit comments