-
Notifications
You must be signed in to change notification settings - Fork 447
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Signed-off-by: Yuan Zhou <[email protected]>
- Loading branch information
Showing
10 changed files
with
2,493 additions
and
2 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
122 changes: 122 additions & 0 deletions
122
shims/spark32/src/main/scala/org/apache/spark/shuffle/SortShuffleWriter.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,122 @@ | ||
/* | ||
* 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.shuffle.sort | ||
|
||
import org.apache.spark._ | ||
import org.apache.spark.internal.{config, Logging} | ||
import org.apache.spark.scheduler.MapStatus | ||
import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleWriteMetricsReporter, ShuffleWriter} | ||
import org.apache.spark.shuffle.api.ShuffleExecutorComponents | ||
import org.apache.spark.util.collection.ExternalSorter | ||
|
||
private[spark] class SortShuffleWriterWrapper[K, V, C]( | ||
handle: BaseShuffleHandle[K, V, C], | ||
mapId: Long, | ||
context: TaskContext, | ||
metrics: ShuffleWriteMetricsReporter, | ||
shuffleExecutorComponents: ShuffleExecutorComponents) | ||
extends ShuffleWriter[K, V] | ||
with Logging { | ||
|
||
private val dep = handle.dependency | ||
|
||
private val blockManager = SparkEnv.get.blockManager | ||
|
||
private var sorter: ExternalSorter[K, V, _] = null | ||
|
||
// Are we in the process of stopping? Because map tasks can call stop() with success = true | ||
// and then call stop() with success = false if they get an exception, we want to make sure | ||
// we don't try deleting files, etc twice. | ||
private var stopping = false | ||
|
||
private var mapStatus: MapStatus = null | ||
|
||
private var partitionLengths: Array[Long] = _ | ||
|
||
private val writeMetrics = context.taskMetrics().shuffleWriteMetrics | ||
|
||
/** Write a bunch of records to this task's output */ | ||
override def write(records: Iterator[Product2[K, V]]): Unit = { | ||
sorter = if (dep.mapSideCombine) { | ||
new ExternalSorter[K, V, C]( | ||
context, | ||
dep.aggregator, | ||
Some(dep.partitioner), | ||
dep.keyOrdering, | ||
dep.serializer) | ||
} else { | ||
// In this case we pass neither an aggregator nor an ordering to the sorter, because we don't | ||
// care whether the keys get sorted in each partition; that will be done on the reduce side | ||
// if the operation being run is sortByKey. | ||
new ExternalSorter[K, V, V]( | ||
context, | ||
aggregator = None, | ||
Some(dep.partitioner), | ||
ordering = None, | ||
dep.serializer) | ||
} | ||
sorter.insertAll(records) | ||
|
||
// Don't bother including the time to open the merged output file in the shuffle write time, | ||
// because it just opens a single file, so is typically too fast to measure accurately | ||
// (see SPARK-3570). | ||
val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( | ||
dep.shuffleId, | ||
mapId, | ||
dep.partitioner.numPartitions) | ||
sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) | ||
partitionLengths = mapOutputWriter.commitAllPartitions(sorter.getChecksums).getPartitionLengths | ||
mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) | ||
} | ||
|
||
/** Close this writer, passing along whether the map completed */ | ||
override def stop(success: Boolean): Option[MapStatus] = { | ||
try { | ||
if (stopping) { | ||
return None | ||
} | ||
stopping = true | ||
if (success) { | ||
return Option(mapStatus) | ||
} else { | ||
return None | ||
} | ||
} finally { | ||
// Clean up our sorter, which may have its own intermediate files | ||
if (sorter != null) { | ||
val startTime = System.nanoTime() | ||
sorter.stop() | ||
writeMetrics.incWriteTime(System.nanoTime - startTime) | ||
sorter = null | ||
} | ||
} | ||
} | ||
|
||
override def getPartitionLengths(): Array[Long] = partitionLengths | ||
} | ||
|
||
private[spark] object SortShuffleWriterWrapper { | ||
def shouldBypassMergeSort(conf: SparkConf, dep: ShuffleDependency[_, _, _]): Boolean = { | ||
// We cannot bypass sorting if we need to do map-side aggregation. | ||
if (dep.mapSideCombine) { | ||
false | ||
} else { | ||
val bypassMergeThreshold: Int = conf.get(config.SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD) | ||
dep.partitioner.numPartitions <= bypassMergeThreshold | ||
} | ||
} | ||
} |
Oops, something went wrong.