zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644423043
##########
File path:
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -290,16 +349,64 @@ void deleteExecutorDirs(Path[] dirs) {
}
}
+ /**
+ * Create StreamCallback for invalid push blocks with the specific error
message.
+ * If specific error message is null, this StreamCallback won't throw
exception in client.
+ */
+ private StreamCallbackWithID createCallbackForInvalidPushBlocks(
+ String streamId,
+ String errorMessage) {
+ return new StreamCallbackWithID() {
+ @Override
+ public String getID() {
+ return streamId;
+ }
+
+ @Override
+ public void onData(String streamId, ByteBuffer buf) {
+ // Ignore the requests. It reaches here either when a request is
received after the
+ // shuffle file is finalized or when a request is for a duplicate
block.
+ }
+
+ @Override
+ public void onComplete(String streamId) {
+ if (errorMessage != null) {
+ // Throw an exception here so the block data is drained from channel
and server
+ // responds RpcFailure to the client.
+ throw new RuntimeException(String.format("Block %s %s", streamId,
errorMessage));
+ }
+ // For duplicate block that is received before the shuffle merge
finalizes, the
+ // server should respond success to the client.
+ }
+
+ @Override
+ public void onFailure(String streamId, Throwable cause) {
+ }
+ };
+ }
+
@Override
public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
// Retrieve merged shuffle file metadata
- AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+ AppAttemptPathsInfo appAttemptPathsInfo =
getAppAttemptPathsInfo(msg.appId);
+ final String streamId = String.format("%s_%d_%d_%d",
+ OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, msg.shuffleId,
msg.mapIndex, msg.reduceId);
+ AppAttemptShuffleId appAttemptShuffleId =
+ new AppAttemptShuffleId(msg.appId, msg.attemptId, msg.shuffleId);
+ if (appAttemptPathsInfo.attemptId != appAttemptShuffleId.attemptId) {
+ // If this Block belongs to a former application attempt, it is
considered late,
+ // as only the blocks from the current application attempt will be merged
+ // TODO: [SPARK-35548] Client should be updated to handle this error.
+ return createCallbackForInvalidPushBlocks(streamId,
Review comment:
Removed from the slim down PR.
##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf,
deleteFilesOnStop: Boolea
}
}
+ /**
+ * Get the list of configured local dirs storing merged shuffle blocks
created by executors
+ * if push based shuffle is enabled. Note that the files in this directory
will be created
+ * by the external shuffle services. We only create the merge_manager
directories and
+ * subdirectories here because currently the shuffle service doesn't have
permission to
+ * create directories under application local directories.
+ */
+ private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf):
Option[Array[File]] = {
+ if (Utils.isPushBasedShuffleEnabled(conf)) {
+ // Will create the merge_manager directory only if it doesn't exist
under any local dir.
+ val localDirs = Utils.getConfiguredLocalDirs(conf)
+ var mergeDirCreated = false;
+ for (rootDir <- localDirs) {
+ val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+ if (mergeDir.exists()) {
+ logDebug(s"Not creating $mergeDir as it already exists")
+ mergeDirCreated = true
+ }
+ }
Review comment:
The original logic:
For loop each local dir, if there is any merge_dir created by other
executors, this executor will not create any local dirs.
Potential issue:
Suppose Executor 1 created the merge_dirs under /tmp/[a-c]. Executor2
launched slightly later, it got local dirs /tmp/[b-d]. Executor2 would not
create merge_dirs in any of them as it found that another executor has created
the merge dir in /tmp/b. But if the executor registration message from
Executor2 gets handled prior to Executor1's, shuffle service will use the
/tmp/[b-d] as the merge dirs. However, the merge_dir under /tmp/d has not been
created by Executor2.
The update logic:
Every Executor should try to create the merge_dir under the local dirs,
which makes sure no matter what ExecutorRegister message gets received in
shuffle service, it is guaranteed that the merge_dir is there with permission
770.
@otterc
##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
hostLocalDirManager = {
if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
- !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+ !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+ Utils.isPushBasedShuffleEnabled(conf)) {
Review comment:
Fixed and added a unit test in HostLocalShuffleReadingSuite to check
whether the hostLocalDirManager gets initiated when push based shuffle is
enabled.
##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
}
}
+ /**
+ * Get the local merged shuffle block data for the given block ID as
multiple chunks.
+ * A merged shuffle file is divided into multiple chunks according to the
index file.
+ * Instead of reading the entire file as a single block, we split it into
smaller chunks
+ * which will be memory efficient when performing certain operations.
+ */
+ def getLocalMergedBlockData(
Review comment:
IndexShuffleBlockResolver will always fetch a local disk, but in
ShuffleBlockResolver, there is no "Local" added in the method name, similarly
to original shuffle fetch. In ShuffleBlockResolver and
IndexShuffleBlockResolver, "Local" is not added to the method name. But in
BlockManager, we have getLocalBlockData and getHostLocalShuffleData.
##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
}
}
+ /**
+ * Get the local merged shuffle block data for the given block ID as
multiple chunks.
+ * A merged shuffle file is divided into multiple chunks according to the
index file.
+ * Instead of reading the entire file as a single block, we split it into
smaller chunks
+ * which will be memory efficient when performing certain operations.
+ */
+ def getLocalMergedBlockData(
Review comment:
Similarly to original shuffle fetch, in ShuffleBlockResolver and
IndexShuffleBlockResolver, "Local" is not added to the method name. But in
BlockManager, we have getLocalBlockData and getHostLocalShuffleData.
##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
}
}
+ /**
+ * Get the local merged shuffle block data for the given block ID as
multiple chunks.
+ * A merged shuffle file is divided into multiple chunks according to the
index file.
+ * Instead of reading the entire file as a single block, we split it into
smaller chunks
+ * which will be memory efficient when performing certain operations.
+ */
+ def getLocalMergedBlockData(
+ blockId: ShuffleBlockId,
+ dirs: Array[String]): Seq[ManagedBuffer] = {
+ shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId, Some(dirs))
+ }
+
+ /**
+ * Get the local merged shuffle block meta data for the given block ID.
+ */
+ def geLocalMergedBlockMeta(
Review comment:
Added Local here, as suggested by @Ngone51.
##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -129,6 +155,9 @@ object BlockId {
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 SHUFFLE_PUSH = "shufflePush_([0-9]+)_([0-9]+)_([0-9]+)".r
+ val SHUFFLE_MERGED = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).data".r
+ val SHUFFLE_MERGED_INDEX =
"shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).index".r
+ val SHUFFLE_MERGED_META =
"shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).meta".r
Review comment:
Just figured it out, this is for unit testing, similarly to other unit
tests for case classes defined in BlockId.scala
##########
File path:
core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
}
}
+ /**
+ * This is only used for reading local merged block data. In such cases, all
chunks in the
+ * merged shuffle file need to be identified at once, so the
ShuffleBlockFetcherIterator
+ * knows how to consume local merged shuffle file as multiple chunks.
+ */
+ override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer]
= {
+ val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId,
blockId.reduceId)
+ val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId,
blockId.reduceId)
+ // Load all the indexes in order to identify all chunks in the specified
merged shuffle file.
+ val size = indexFile.length.toInt
+ val buffer = ByteBuffer.allocate(size)
+ val offsets = buffer.asLongBuffer
+ val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+ try {
+ dis.readFully(buffer.array)
+ } finally {
+ dis.close()
+ }
Review comment:
Updated as @mridulm suggested.
##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
dir.getCanonicalFile
}
+ /**
+ * Create a directory that is writable by the group.
+ * Grant the customized permission so the shuffle server can
+ * create subdirs/files within the merge folder.
Review comment:
Moved to DiskBlockManager. Also need to change the access modifiers for
MAX_DIR_CREATION_ATTEMPTS in Utils.scala.
##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -17,12 +17,13 @@
package org.apache.spark.util
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput,
DataOutputStream, File,
- FileOutputStream, PrintStream, SequenceInputStream}
+import java.io._
Review comment:
You did recommend this change in former reviews.
https://github.com/apache/spark/pull/32007#discussion_r622840432
##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
}
/**
- * Push based shuffle can only be enabled when external shuffle service is
enabled.
+ * Push based shuffle can only be enabled when the application is submitted
+ * to run in YARN mode, with external shuffle service enabled and
+ * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to
1.
+ * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
*/
def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
- (conf.get(IS_TESTING).getOrElse(false) ||
conf.get(SHUFFLE_SERVICE_ENABLED))
+ (conf.get(IS_TESTING).getOrElse(false) ||
+ (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+ conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&
+ getYarnMaxAttempts(conf) == 1)
+ }
+
+ /** Returns the maximum number of attempts to register the AM in YARN mode.
*/
+ def getYarnMaxAttempts(conf: SparkConf): Int = {
+ val sparkMaxAttempts =
conf.getOption("spark.yarn.maxAttempts").map(_.toInt)
+ val yarnMaxAttempts = getSparkOrYarnConfig(conf,
YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+ YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS.toString).toInt
+ sparkMaxAttempts match {
+ case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
+ case None => yarnMaxAttempts
+ }
Review comment:
The code does exist in Spark code base, but in the module of
resource-manager/yarn. Should we move this part to Utils.scala and then call it
in YarnRMClient? Or we can just make a code duplicate here?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]