Skip to content

Commit 6906b69

Browse files
mateizrxin
authored andcommitted
SPARK-2787: Make sort-based shuffle write files directly when there's no sorting/aggregation and # partitions is small
As described in https://issues.apache.org/jira/browse/SPARK-2787, right now sort-based shuffle is more expensive than hash-based for map operations that do no partial aggregation or sorting, such as groupByKey. This is because it has to serialize each data item twice (once when spilling to intermediate files, and then again when merging these files object-by-object). This patch adds a code path to just write separate files directly if the # of output partitions is small, and concatenate them at the end to produce a sorted file. On the unit test side, I added some tests that force or don't force this bypass path to be used, and checked that our tests for other features (e.g. all the operations) cover both cases. Author: Matei Zaharia <matei@databricks.com> Closes #1799 from mateiz/SPARK-2787 and squashes the following commits: 88cf26a [Matei Zaharia] Fix rebase 10233af [Matei Zaharia] Review comments 398cb95 [Matei Zaharia] Fix looking up shuffle manager in conf ca3efd9 [Matei Zaharia] Add docs for shuffle manager properties, and allow short names for them d0ae3c5 [Matei Zaharia] Fix some comments 90d084f [Matei Zaharia] Add code path to bypass merge-sort in ExternalSorter, and tests 31e5d7c [Matei Zaharia] Move existing logic for writing partitioned files into ExternalSorter
1 parent 32096c2 commit 6906b69

File tree

6 files changed

+407
-118
lines changed

6 files changed

+407
-118
lines changed

core/src/main/scala/org/apache/spark/SparkEnv.scala

Lines changed: 18 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -156,11 +156,9 @@ object SparkEnv extends Logging {
156156
conf.set("spark.driver.port", boundPort.toString)
157157
}
158158

159-
// Create an instance of the class named by the given Java system property, or by
160-
// defaultClassName if the property is not set, and return it as a T
161-
def instantiateClass[T](propertyName: String, defaultClassName: String): T = {
162-
val name = conf.get(propertyName, defaultClassName)
163-
val cls = Class.forName(name, true, Utils.getContextOrSparkClassLoader)
159+
// Create an instance of the class with the given name, possibly initializing it with our conf
160+
def instantiateClass[T](className: String): T = {
161+
val cls = Class.forName(className, true, Utils.getContextOrSparkClassLoader)
164162
// Look for a constructor taking a SparkConf and a boolean isDriver, then one taking just
165163
// SparkConf, then one taking no arguments
166164
try {
@@ -178,11 +176,17 @@ object SparkEnv extends Logging {
178176
}
179177
}
180178

181-
val serializer = instantiateClass[Serializer](
179+
// Create an instance of the class named by the given SparkConf property, or defaultClassName
180+
// if the property is not set, possibly initializing it with our conf
181+
def instantiateClassFromConf[T](propertyName: String, defaultClassName: String): T = {
182+
instantiateClass[T](conf.get(propertyName, defaultClassName))
183+
}
184+
185+
val serializer = instantiateClassFromConf[Serializer](
182186
"spark.serializer", "org.apache.spark.serializer.JavaSerializer")
183187
logDebug(s"Using serializer: ${serializer.getClass}")
184188

185-
val closureSerializer = instantiateClass[Serializer](
189+
val closureSerializer = instantiateClassFromConf[Serializer](
186190
"spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")
187191

188192
def registerOrLookup(name: String, newActor: => Actor): ActorRef = {
@@ -246,8 +250,13 @@ object SparkEnv extends Logging {
246250
"."
247251
}
248252

249-
val shuffleManager = instantiateClass[ShuffleManager](
250-
"spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager")
253+
// Let the user specify short names for shuffle managers
254+
val shortShuffleMgrNames = Map(
255+
"hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager",
256+
"sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager")
257+
val shuffleMgrName = conf.get("spark.shuffle.manager", "hash")
258+
val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName)
259+
val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass)
251260

252261
val shuffleMemoryManager = new ShuffleMemoryManager(conf)
253262

core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ private[spark] class HashShuffleReader[K, C](
5858
// Create an ExternalSorter to sort the data. Note that if spark.shuffle.spill is disabled,
5959
// the ExternalSorter won't spill to disk.
6060
val sorter = new ExternalSorter[K, C, C](ordering = Some(keyOrd), serializer = Some(ser))
61-
sorter.write(aggregatedIter)
61+
sorter.insertAll(aggregatedIter)
6262
context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled
6363
context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled
6464
sorter.iterator

core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala

Lines changed: 21 additions & 59 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@ private[spark] class SortShuffleWriter[K, V, C](
4444

4545
private var sorter: ExternalSorter[K, V, _] = null
4646
private var outputFile: File = null
47+
private var indexFile: File = null
4748

4849
// Are we in the process of stopping? Because map tasks can call stop() with success = true
4950
// and then call stop() with success = false if they get an exception, we want to make sure
@@ -57,78 +58,36 @@ private[spark] class SortShuffleWriter[K, V, C](
5758

5859
/** Write a bunch of records to this task's output */
5960
override def write(records: Iterator[_ <: Product2[K, V]]): Unit = {
60-
// Get an iterator with the elements for each partition ID
61-
val partitions: Iterator[(Int, Iterator[Product2[K, _]])] = {
62-
if (dep.mapSideCombine) {
63-
if (!dep.aggregator.isDefined) {
64-
throw new IllegalStateException("Aggregator is empty for map-side combine")
65-
}
66-
sorter = new ExternalSorter[K, V, C](
67-
dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer)
68-
sorter.write(records)
69-
sorter.partitionedIterator
70-
} else {
71-
// In this case we pass neither an aggregator nor an ordering to the sorter, because we
72-
// don't care whether the keys get sorted in each partition; that will be done on the
73-
// reduce side if the operation being run is sortByKey.
74-
sorter = new ExternalSorter[K, V, V](
75-
None, Some(dep.partitioner), None, dep.serializer)
76-
sorter.write(records)
77-
sorter.partitionedIterator
61+
if (dep.mapSideCombine) {
62+
if (!dep.aggregator.isDefined) {
63+
throw new IllegalStateException("Aggregator is empty for map-side combine")
7864
}
65+
sorter = new ExternalSorter[K, V, C](
66+
dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer)
67+
sorter.insertAll(records)
68+
} else {
69+
// In this case we pass neither an aggregator nor an ordering to the sorter, because we don't
70+
// care whether the keys get sorted in each partition; that will be done on the reduce side
71+
// if the operation being run is sortByKey.
72+
sorter = new ExternalSorter[K, V, V](
73+
None, Some(dep.partitioner), None, dep.serializer)
74+
sorter.insertAll(records)
7975
}
8076

8177
// Create a single shuffle file with reduce ID 0 that we'll write all results to. We'll later
8278
// serve different ranges of this file using an index file that we create at the end.
8379
val blockId = ShuffleBlockId(dep.shuffleId, mapId, 0)
84-
outputFile = blockManager.diskBlockManager.getFile(blockId)
85-
86-
// Track location of each range in the output file
87-
val offsets = new Array[Long](numPartitions + 1)
88-
val lengths = new Array[Long](numPartitions)
89-
90-
for ((id, elements) <- partitions) {
91-
if (elements.hasNext) {
92-
val writer = blockManager.getDiskWriter(blockId, outputFile, ser, fileBufferSize,
93-
writeMetrics)
94-
for (elem <- elements) {
95-
writer.write(elem)
96-
}
97-
writer.commitAndClose()
98-
val segment = writer.fileSegment()
99-
offsets(id + 1) = segment.offset + segment.length
100-
lengths(id) = segment.length
101-
} else {
102-
// The partition is empty; don't create a new writer to avoid writing headers, etc
103-
offsets(id + 1) = offsets(id)
104-
}
105-
}
106-
107-
context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled
108-
context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled
10980

110-
// Write an index file with the offsets of each block, plus a final offset at the end for the
111-
// end of the output file. This will be used by SortShuffleManager.getBlockLocation to figure
112-
// out where each block begins and ends.
81+
outputFile = blockManager.diskBlockManager.getFile(blockId)
82+
indexFile = blockManager.diskBlockManager.getFile(blockId.name + ".index")
11383

114-
val diskBlockManager = blockManager.diskBlockManager
115-
val indexFile = diskBlockManager.getFile(blockId.name + ".index")
116-
val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile)))
117-
try {
118-
var i = 0
119-
while (i < numPartitions + 1) {
120-
out.writeLong(offsets(i))
121-
i += 1
122-
}
123-
} finally {
124-
out.close()
125-
}
84+
val partitionLengths = sorter.writePartitionedFile(blockId, context)
12685

12786
// Register our map output with the ShuffleBlockManager, which handles cleaning it over time
12887
blockManager.shuffleBlockManager.addCompletedMap(dep.shuffleId, mapId, numPartitions)
12988

13089
mapStatus = new MapStatus(blockManager.blockManagerId,
131-
lengths.map(MapOutputTracker.compressSize))
90+
partitionLengths.map(MapOutputTracker.compressSize))
13291
}
13392

13493
/** Close this writer, passing along whether the map completed */
@@ -145,6 +104,9 @@ private[spark] class SortShuffleWriter[K, V, C](
145104
if (outputFile != null) {
146105
outputFile.delete()
147106
}
107+
if (indexFile != null) {
108+
indexFile.delete()
109+
}
148110
return None
149111
}
150112
} finally {

0 commit comments

Comments
 (0)