-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-8029] Robust shuffle writer #9610
Changes from 4 commits
b2a90c6
9f0d2f9
55485a9
6deccff
f0c2a5d
d0b937f
35bd469
71b12bf
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 |
---|---|---|
|
@@ -21,13 +21,12 @@ import java.io._ | |
|
||
import com.google.common.io.ByteStreams | ||
|
||
import org.apache.spark.{SparkConf, SparkEnv, Logging} | ||
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} | ||
import org.apache.spark.network.netty.SparkTransportConf | ||
import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID | ||
import org.apache.spark.storage._ | ||
import org.apache.spark.util.Utils | ||
|
||
import IndexShuffleBlockResolver.NOOP_REDUCE_ID | ||
import org.apache.spark.{Logging, SparkConf, SparkEnv} | ||
|
||
/** | ||
* Create and maintain the shuffle blocks' mapping between logic block and physical file location. | ||
|
@@ -79,9 +78,10 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB | |
* end of the output file. This will be used by getBlockData to figure out where each block | ||
* begins and ends. | ||
* */ | ||
def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): Unit = { | ||
def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long], dataTmp: File): Unit = { | ||
val indexFile = getIndexFile(shuffleId, mapId) | ||
val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) | ||
val indexTmp = Utils.tempFileWith(indexFile) | ||
val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp))) | ||
Utils.tryWithSafeFinally { | ||
// We take in lengths of each block, need to convert it to offsets. | ||
var offset = 0L | ||
|
@@ -93,6 +93,29 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB | |
} { | ||
out.close() | ||
} | ||
|
||
val dataFile = getDataFile(shuffleId, mapId) | ||
synchronized { | ||
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. Can you add a big comment explaining what is going on here? Also worth noting that there is only one 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. is this |
||
if (dataFile.exists() && indexFile.exists()) { | ||
if (dataTmp != null && dataTmp.exists()) { | ||
dataTmp.delete() | ||
} | ||
indexTmp.delete() | ||
} else { | ||
if (indexFile.exists()) { | ||
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.
|
||
indexFile.delete() | ||
} | ||
if (!indexTmp.renameTo(indexFile)) { | ||
throw new IOException("fail to rename data file " + indexTmp + " to " + indexFile) | ||
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. data -> index |
||
} | ||
if (dataFile.exists()) { | ||
dataFile.delete() | ||
} | ||
if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) { | ||
throw new IOException("fail to rename data file " + dataTmp + " to " + dataFile) | ||
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. I don't think there is a particular flaw here, but its a bit hard to follow since its a mix of first-attempt-wins and last-attempt wins. First attempt if there is a data file & index file; last attempt if its only an index file. the problem w/ last-attempt is that this delete will fail on windows if the file is open for reading, I believe. Though we can't get around that because SPARK-4085 always requires us to delete some files that might be open, in which case we hope that we don't run into this race again on the next retry. It would be nice to minimize that case, though. We'd be closer to first-attempt-wins if we always wrote a dataFile, even if its empty when dataTmp == null. There is also an issue w/ mapStatus & non-deterministic data. It might not matter which output you get, but the mapstatus should be consistent with the data that is read. If attempt 1 writes non-empty outputs a,b,c, and attempt 2 writes non-empty outputs d,e,f (which are not committed), the reduce tasks might get the mapstatus for attempt 2, look for outputs d,e,f, and get nothing but empty blocks. Matei had suggested writing the mapstatus to a file, so that subsequent attempts always return the map status corresponding to the first successful attempt. |
||
} | ||
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. should we check
what happens if this fails? Should we throw an exception? 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. This will slowdown the normal path, I think it's not needed. |
||
} | ||
} | ||
} | ||
|
||
override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,8 @@ | |
|
||
package org.apache.spark.shuffle.hash | ||
|
||
import java.io.IOException | ||
|
||
import org.apache.spark._ | ||
import org.apache.spark.executor.ShuffleWriteMetrics | ||
import org.apache.spark.scheduler.MapStatus | ||
|
@@ -106,6 +108,19 @@ private[spark] class HashShuffleWriter[K, V]( | |
writer.commitAndClose() | ||
writer.fileSegment().length | ||
} | ||
// rename all shuffle files to final paths | ||
shuffle.writers.zip(sizes).foreach { case (writer: DiskBlockObjectWriter, size: Long) => | ||
if (size > 0) { | ||
val output = blockManager.diskBlockManager.getFile(writer.blockId) | ||
if (output.exists()) { | ||
writer.file.delete() | ||
} else { | ||
if (!writer.file.renameTo(output)) { | ||
throw new IOException(s"fail to rename ${writer.file} to $output") | ||
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. same problem here on partially existing shuffle output. Also, the 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. I think these partitions are independent, they should be OK whenever it's generated in different attempt, or that's the basic idea of how RDD works (could be re-run and got the same result). If not, for example, the items in RDD is random are generated randomly, then it also does not matter if it's different across attempts. 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. The 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. yeah I suppose it all depends on what the model is for non-deterministic data. The reduce tasks can read data from a mix attempts, but I guess that is OK (we can't completely prevent it in any case). There is also the problem of returning the right mapstatus here, but it doesn't matter as much in this case -- you will at least return some set of non-empty blocks that is consistent with the shuffle data on disk, even if the sizes can be arbitrarily wrong. Also I know its super-rare, but there is a race between I also find it a weird that this is neither first or last attempt wins -- the first attempt to get to each output file wins, but it can be a mix of attempts. again I'd include a comment explaining the logic |
||
} | ||
} | ||
} | ||
} | ||
MapStatus(blockManager.shuffleServerId, sizes) | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,8 +20,9 @@ package org.apache.spark.shuffle.sort | |
import org.apache.spark._ | ||
import org.apache.spark.executor.ShuffleWriteMetrics | ||
import org.apache.spark.scheduler.MapStatus | ||
import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriter, BaseShuffleHandle} | ||
import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} | ||
import org.apache.spark.storage.ShuffleBlockId | ||
import org.apache.spark.util.Utils | ||
import org.apache.spark.util.collection.ExternalSorter | ||
|
||
private[spark] class SortShuffleWriter[K, V, C]( | ||
|
@@ -65,11 +66,11 @@ private[spark] class SortShuffleWriter[K, V, C]( | |
// 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 outputFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) | ||
val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) | ||
val tmp = Utils.tempFileWith(output) | ||
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. can you call these 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. This is only one file |
||
val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) | ||
val partitionLengths = sorter.writePartitionedFile(blockId, outputFile) | ||
shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths) | ||
|
||
val partitionLengths = sorter.writePartitionedFile(blockId, tmp) | ||
shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths, tmp) | ||
mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) | ||
} | ||
|
||
|
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.
why not use your new method
Utils.withTempFile