-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-9853][Core] Optimize shuffle fetch of contiguous partition IDs #19788
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
e947bcb
53affd4
e437a26
12163f3
7aa805b
9cb1f0f
2799886
80c8da9
b410f9c
85a4f49
c7db28d
ff4bece
80bb53c
e18eb59
69cde07
978e10d
2a1a97c
d8691ee
f30f057
6c684bc
a9ab62e
87c9af6
7fa0fb9
ab4aa5f
e5e7c38
6ee294a
583666b
c133776
fc0fe77
7009a5e
4a31e9c
08d2ca1
5efa1af
3412bcb
63d9eb1
a0cd415
3da48d8
c650e65
752f90c
b91c3e9
9ffe59d
9354ed0
0f49142
535fe5f
087422f
2191f8d
2ca4092
6496abf
e91dbd5
6cb7110
05e4465
eb752aa
694ec2d
45d7096
4850d33
28355b1
7a65cc4
a9ffdee
96b0082
ace48fd
86aa1fb
fcf434b
a360bbf
2f28a7e
5bd6d73
aa6134b
c6ebe0e
be54f49
1c91d20
f810e28
8a1815f
076894f
b33b4dd
91aeff9
ea1795a
dd980dd
e9d8620
5933bf8
57fab14
c75e016
8398120
2424be0
bd9f70e
5e4430a
401bddb
3d4fc7e
039ae85
92c0ab6
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -812,10 +812,13 @@ private[spark] object MapOutputTracker extends Logging { | |
| logError(errorMessage) | ||
| throw new MetadataFetchFailedException(shuffleId, startPartition, errorMessage) | ||
| } else { | ||
| var totalSize = 0L | ||
| for (part <- startPartition until endPartition) { | ||
| splitsByAddress.getOrElseUpdate(status.location, ArrayBuffer()) += | ||
| ((ShuffleBlockId(shuffleId, mapId, part), status.getSizeForBlock(part))) | ||
| totalSize += status.getSizeForBlock(part) | ||
| } | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| splitsByAddress.getOrElseUpdate(status.location, ArrayBuffer()) += | ||
| ((ShuffleBlockId(shuffleId, mapId, startPartition, endPartition - startPartition), | ||
| totalSize)) | ||
|
||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -196,12 +196,14 @@ private[spark] class IndexShuffleBlockResolver( | |
| override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { | ||
| // The block is actually going to be a range of a single map output file for this map, so | ||
| // find out the consolidated file, then the offset within that from our index | ||
| logDebug(s"Fetch block data for $blockId") | ||
|
||
| val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId) | ||
|
|
||
| val in = new DataInputStream(Files.newInputStream(indexFile.toPath)) | ||
| try { | ||
| ByteStreams.skipFully(in, blockId.reduceId * 8) | ||
| val offset = in.readLong() | ||
| ByteStreams.skipFully(in, (blockId.length - 1) * 8) | ||
|
||
| val nextOffset = in.readLong() | ||
| new FileSegmentManagedBuffer( | ||
| transportConf, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -52,8 +52,9 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { | |
| // Format of the shuffle block ids (including data and index) should be kept in sync with | ||
| // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getBlockData(). | ||
| @DeveloperApi | ||
| case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { | ||
| override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId | ||
| case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int, length: Int = 1) | ||
| extends BlockId { | ||
| override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + length | ||
|
||
| } | ||
|
|
||
| @DeveloperApi | ||
|
|
@@ -103,7 +104,7 @@ class UnrecognizedBlockId(name: String) | |
| @DeveloperApi | ||
| object BlockId { | ||
| val RDD = "rdd_([0-9]+)_([0-9]+)".r | ||
| val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r | ||
| val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r | ||
| val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r | ||
| val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r | ||
| val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r | ||
|
|
@@ -116,8 +117,8 @@ object BlockId { | |
| def apply(name: String): BlockId = name match { | ||
| case RDD(rddId, splitIndex) => | ||
| RDDBlockId(rddId.toInt, splitIndex.toInt) | ||
| case SHUFFLE(shuffleId, mapId, reduceId) => | ||
| ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) | ||
| case SHUFFLE(shuffleId, mapId, reduceId, n) => | ||
|
||
| ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, n.toInt) | ||
| case SHUFFLE_DATA(shuffleId, mapId, reduceId) => | ||
| ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) | ||
| case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This can be simplified like:
val totalSize = (startPartition until endPartition).map(status.getSizeForXXX).sum.