Skip to content

Commit b71c130

Browse files
attilapirosMarcelo Vanzin
authored and
Marcelo Vanzin
committed
[SPARK-27622][CORE] Avoiding the network when block manager fetches disk persisted RDD blocks from the same host
## What changes were proposed in this pull request? Before this PR during fetching a disk persisted RDD block the network was always used to get the requested block content even when both the source and fetcher executor was running on the same host. The idea to access another executor local disk files by directly reading the disk comes from the external shuffle service where the local dirs are stored for each executor (block manager). To make this possible the following changes are done: - `RegisterBlockManager` message is extended with the `localDirs` which is stored by the block manager master for each block manager as a new property of the `BlockManagerInfo` - `GetLocationsAndStatus` is extended with the requester host - `BlockLocationsAndStatus` (the reply for `GetLocationsAndStatus` message) is extended with the an option of local directories, which is filled with a local directories of a same host executor (if there is any, otherwise None is used). This is where the block content can be read from. Shuffle blocks are out of scope of this PR: there will be a separate PR opened for that (for another Jira issue). ## How was this patch tested? With a new unit test in `BlockManagerSuite`. See the the test prefixed by "SPARK-27622: avoid the network when block requested from same host". Closes apache#24554 from attilapiros/SPARK-27622. Authored-by: “attilapiros” <[email protected]> Signed-off-by: Marcelo Vanzin <[email protected]>
1 parent 1a3858a commit b71c130

File tree

11 files changed

+342
-93
lines changed

11 files changed

+342
-93
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,66 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.network.shuffle;
19+
20+
import java.io.File;
21+
import java.util.regex.Matcher;
22+
import java.util.regex.Pattern;
23+
24+
import com.google.common.annotations.VisibleForTesting;
25+
26+
import org.apache.spark.network.util.JavaUtils;
27+
28+
public class ExecutorDiskUtils {
29+
30+
private static final Pattern MULTIPLE_SEPARATORS = Pattern.compile(File.separator + "{2,}");
31+
32+
/**
33+
* Hashes a filename into the corresponding local directory, in a manner consistent with
34+
* Spark's DiskBlockManager.getFile().
35+
*/
36+
public static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) {
37+
int hash = JavaUtils.nonNegativeHash(filename);
38+
String localDir = localDirs[hash % localDirs.length];
39+
int subDirId = (hash / localDirs.length) % subDirsPerLocalDir;
40+
return new File(createNormalizedInternedPathname(
41+
localDir, String.format("%02x", subDirId), filename));
42+
}
43+
44+
/**
45+
* This method is needed to avoid the situation when multiple File instances for the
46+
* same pathname "foo/bar" are created, each with a separate copy of the "foo/bar" String.
47+
* According to measurements, in some scenarios such duplicate strings may waste a lot
48+
* of memory (~ 10% of the heap). To avoid that, we intern the pathname, and before that
49+
* we make sure that it's in a normalized form (contains no "//", "///" etc.) Otherwise,
50+
* the internal code in java.io.File would normalize it later, creating a new "foo/bar"
51+
* String copy. Unfortunately, we cannot just reuse the normalization code that java.io.File
52+
* uses, since it is in the package-private class java.io.FileSystem.
53+
*/
54+
@VisibleForTesting
55+
static String createNormalizedInternedPathname(String dir1, String dir2, String fname) {
56+
String pathname = dir1 + File.separator + dir2 + File.separator + fname;
57+
Matcher m = MULTIPLE_SEPARATORS.matcher(pathname);
58+
pathname = m.replaceAll("/");
59+
// A single trailing slash needs to be taken care of separately
60+
if (pathname.length() > 1 && pathname.endsWith("/")) {
61+
pathname = pathname.substring(0, pathname.length() - 1);
62+
}
63+
return pathname.intern();
64+
}
65+
66+
}

common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java

+5-40
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@
2424
import java.util.concurrent.ExecutionException;
2525
import java.util.concurrent.Executor;
2626
import java.util.concurrent.Executors;
27-
import java.util.regex.Matcher;
2827
import java.util.regex.Pattern;
2928

3029
import com.fasterxml.jackson.annotation.JsonCreator;
@@ -298,15 +297,15 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) {
298297
*/
299298
private ManagedBuffer getSortBasedShuffleBlockData(
300299
ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) {
301-
File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir,
300+
File indexFile = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir,
302301
"shuffle_" + shuffleId + "_" + mapId + "_0.index");
303302

304303
try {
305304
ShuffleIndexInformation shuffleIndexInformation = shuffleIndexCache.get(indexFile);
306305
ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(reduceId);
307306
return new FileSegmentManagedBuffer(
308307
conf,
309-
getFile(executor.localDirs, executor.subDirsPerLocalDir,
308+
ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir,
310309
"shuffle_" + shuffleId + "_" + mapId + "_0.data"),
311310
shuffleIndexRecord.getOffset(),
312311
shuffleIndexRecord.getLength());
@@ -317,7 +316,7 @@ private ManagedBuffer getSortBasedShuffleBlockData(
317316

318317
public ManagedBuffer getDiskPersistedRddBlockData(
319318
ExecutorShuffleInfo executor, int rddId, int splitIndex) {
320-
File file = getFile(executor.localDirs, executor.subDirsPerLocalDir,
319+
File file = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir,
321320
"rdd_" + rddId + "_" + splitIndex);
322321
long fileLength = file.length();
323322
ManagedBuffer res = null;
@@ -327,19 +326,6 @@ public ManagedBuffer getDiskPersistedRddBlockData(
327326
return res;
328327
}
329328

330-
/**
331-
* Hashes a filename into the corresponding local directory, in a manner consistent with
332-
* Spark's DiskBlockManager.getFile().
333-
*/
334-
@VisibleForTesting
335-
static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) {
336-
int hash = JavaUtils.nonNegativeHash(filename);
337-
String localDir = localDirs[hash % localDirs.length];
338-
int subDirId = (hash / localDirs.length) % subDirsPerLocalDir;
339-
return new File(createNormalizedInternedPathname(
340-
localDir, String.format("%02x", subDirId), filename));
341-
}
342-
343329
void close() {
344330
if (db != null) {
345331
try {
@@ -350,28 +336,6 @@ void close() {
350336
}
351337
}
352338

353-
/**
354-
* This method is needed to avoid the situation when multiple File instances for the
355-
* same pathname "foo/bar" are created, each with a separate copy of the "foo/bar" String.
356-
* According to measurements, in some scenarios such duplicate strings may waste a lot
357-
* of memory (~ 10% of the heap). To avoid that, we intern the pathname, and before that
358-
* we make sure that it's in a normalized form (contains no "//", "///" etc.) Otherwise,
359-
* the internal code in java.io.File would normalize it later, creating a new "foo/bar"
360-
* String copy. Unfortunately, we cannot just reuse the normalization code that java.io.File
361-
* uses, since it is in the package-private class java.io.FileSystem.
362-
*/
363-
@VisibleForTesting
364-
static String createNormalizedInternedPathname(String dir1, String dir2, String fname) {
365-
String pathname = dir1 + File.separator + dir2 + File.separator + fname;
366-
Matcher m = MULTIPLE_SEPARATORS.matcher(pathname);
367-
pathname = m.replaceAll("/");
368-
// A single trailing slash needs to be taken care of separately
369-
if (pathname.length() > 1 && pathname.endsWith("/")) {
370-
pathname = pathname.substring(0, pathname.length() - 1);
371-
}
372-
return pathname.intern();
373-
}
374-
375339
public int removeBlocks(String appId, String execId, String[] blockIds) {
376340
ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId));
377341
if (executor == null) {
@@ -380,7 +344,8 @@ public int removeBlocks(String appId, String execId, String[] blockIds) {
380344
}
381345
int numRemovedBlocks = 0;
382346
for (String blockId : blockIds) {
383-
File file = getFile(executor.localDirs, executor.subDirsPerLocalDir, blockId);
347+
File file =
348+
ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, blockId);
384349
if (file.delete()) {
385350
numRemovedBlocks++;
386351
} else {

common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -149,7 +149,7 @@ public void testNormalizeAndInternPathname() {
149149

150150
private void assertPathsMatch(String p1, String p2, String p3, String expectedPathname) {
151151
String normPathname =
152-
ExternalShuffleBlockResolver.createNormalizedInternedPathname(p1, p2, p3);
152+
ExecutorDiskUtils.createNormalizedInternedPathname(p1, p2, p3);
153153
assertEquals(expectedPathname, normPathname);
154154
File file = new File(normPathname);
155155
String returnedPath = file.getPath();

common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java

+5-4
Original file line numberDiff line numberDiff line change
@@ -76,9 +76,9 @@ public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) thr
7676

7777
try {
7878
dataStream = new FileOutputStream(
79-
ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".data"));
79+
ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, blockId + ".data"));
8080
indexStream = new DataOutputStream(new FileOutputStream(
81-
ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".index")));
81+
ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, blockId + ".index")));
8282

8383
long offset = 0;
8484
indexStream.writeLong(offset);
@@ -121,10 +121,11 @@ private void insertFile(String filename) throws IOException {
121121

122122
private void insertFile(String filename, byte[] block) throws IOException {
123123
OutputStream dataStream = null;
124-
File file = ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, filename);
124+
File file = ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, filename);
125125
assert(!file.exists()) : "this test file has been already generated";
126126
try {
127-
dataStream = new FileOutputStream(file);
127+
dataStream = new FileOutputStream(
128+
ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, filename));
128129
dataStream.write(block);
129130
} finally {
130131
Closeables.close(dataStream, false);

core/src/main/scala/org/apache/spark/storage/BlockManager.scala

+98-23
Original file line numberDiff line numberDiff line change
@@ -43,8 +43,9 @@ import org.apache.spark.internal.config.Network
4343
import org.apache.spark.memory.{MemoryManager, MemoryMode}
4444
import org.apache.spark.metrics.source.Source
4545
import org.apache.spark.network._
46-
import org.apache.spark.network.buffer.ManagedBuffer
46+
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
4747
import org.apache.spark.network.client.StreamCallbackWithID
48+
import org.apache.spark.network.netty.SparkTransportConf
4849
import org.apache.spark.network.shuffle._
4950
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
5051
import org.apache.spark.network.util.TransportConf
@@ -138,6 +139,8 @@ private[spark] class BlockManager(
138139
private val remoteReadNioBufferConversion =
139140
conf.get(Network.NETWORK_REMOTE_READ_NIO_BUFFER_CONVERSION)
140141

142+
private[spark] val subDirsPerLocalDir = conf.get(config.DISKSTORE_SUB_DIRECTORIES)
143+
141144
val diskBlockManager = {
142145
// Only perform cleanup if an external service is not serving our shuffle files.
143146
val deleteFilesOnStop =
@@ -411,6 +414,7 @@ private[spark] class BlockManager(
411414

412415
val idFromMaster = master.registerBlockManager(
413416
id,
417+
diskBlockManager.localDirsString,
414418
maxOnHeapMemory,
415419
maxOffHeapMemory,
416420
slaveEndpoint)
@@ -445,7 +449,7 @@ private[spark] class BlockManager(
445449
private def registerWithExternalShuffleServer() {
446450
logInfo("Registering executor with local external shuffle service.")
447451
val shuffleConfig = new ExecutorShuffleInfo(
448-
diskBlockManager.localDirs.map(_.toString),
452+
diskBlockManager.localDirsString,
449453
diskBlockManager.subDirsPerLocalDir,
450454
shuffleManager.getClass.getName)
451455

@@ -500,7 +504,8 @@ private[spark] class BlockManager(
500504
def reregister(): Unit = {
501505
// TODO: We might need to rate limit re-registering.
502506
logInfo(s"BlockManager $blockManagerId re-registering with master")
503-
master.registerBlockManager(blockManagerId, maxOnHeapMemory, maxOffHeapMemory, slaveEndpoint)
507+
master.registerBlockManager(blockManagerId, diskBlockManager.localDirsString, maxOnHeapMemory,
508+
maxOffHeapMemory, slaveEndpoint)
504509
reportAllBlocks()
505510
}
506511

@@ -827,10 +832,63 @@ private[spark] class BlockManager(
827832
*/
828833
private[spark] def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = {
829834
val ct = implicitly[ClassTag[T]]
830-
getRemoteManagedBuffer(blockId).map { data =>
835+
getRemoteBlock(blockId, (data: ManagedBuffer) => {
831836
val values =
832837
serializerManager.dataDeserializeStream(blockId, data.createInputStream())(ct)
833838
new BlockResult(values, DataReadMethod.Network, data.size)
839+
})
840+
}
841+
842+
/**
843+
* Get the remote block and transform it to the provided data type.
844+
*
845+
* If the block is persisted to the disk and stored at an executor running on the same host then
846+
* first it is tried to be accessed using the local directories of the other executor directly.
847+
* If the file is successfully identified then tried to be transformed by the provided
848+
* transformation function which expected to open the file. If there is any exception during this
849+
* transformation then block access falls back to fetching it from the remote executor via the
850+
* network.
851+
*
852+
* @param blockId identifies the block to get
853+
* @param bufferTransformer this transformer expected to open the file if the block is backed by a
854+
* file by this it is guaranteed the whole content can be loaded
855+
* @tparam T result type
856+
* @return
857+
*/
858+
private[spark] def getRemoteBlock[T](
859+
blockId: BlockId,
860+
bufferTransformer: ManagedBuffer => T): Option[T] = {
861+
logDebug(s"Getting remote block $blockId")
862+
require(blockId != null, "BlockId is null")
863+
864+
// Because all the remote blocks are registered in driver, it is not necessary to ask
865+
// all the slave executors to get block status.
866+
val locationsAndStatusOption = master.getLocationsAndStatus(blockId, blockManagerId.host)
867+
if (locationsAndStatusOption.isEmpty) {
868+
logDebug(s"Block $blockId is unknown by block manager master")
869+
None
870+
} else {
871+
val locationsAndStatus = locationsAndStatusOption.get
872+
val blockSize = locationsAndStatus.status.diskSize.max(locationsAndStatus.status.memSize)
873+
874+
locationsAndStatus.localDirs.flatMap { localDirs =>
875+
val blockDataOption =
876+
readDiskBlockFromSameHostExecutor(blockId, localDirs, locationsAndStatus.status.diskSize)
877+
val res = blockDataOption.flatMap { blockData =>
878+
try {
879+
Some(bufferTransformer(blockData))
880+
} catch {
881+
case NonFatal(e) =>
882+
logDebug("Block from the same host executor cannot be opened: ", e)
883+
None
884+
}
885+
}
886+
logInfo(s"Read $blockId from the disk of a same host executor is " +
887+
(if (res.isDefined) "successful." else "failed."))
888+
res
889+
}.orElse {
890+
fetchRemoteManagedBuffer(blockId, blockSize, locationsAndStatus).map(bufferTransformer)
891+
}
834892
}
835893
}
836894

@@ -861,22 +919,12 @@ private[spark] class BlockManager(
861919
}
862920

863921
/**
864-
* Get block from remote block managers as a ManagedBuffer.
922+
* Fetch the block from remote block managers as a ManagedBuffer.
865923
*/
866-
private def getRemoteManagedBuffer(blockId: BlockId): Option[ManagedBuffer] = {
867-
logDebug(s"Getting remote block $blockId")
868-
require(blockId != null, "BlockId is null")
869-
var runningFailureCount = 0
870-
var totalFailureCount = 0
871-
872-
// Because all the remote blocks are registered in driver, it is not necessary to ask
873-
// all the slave executors to get block status.
874-
val locationsAndStatus = master.getLocationsAndStatus(blockId)
875-
val blockSize = locationsAndStatus.map { b =>
876-
b.status.diskSize.max(b.status.memSize)
877-
}.getOrElse(0L)
878-
val blockLocations = locationsAndStatus.map(_.locations).getOrElse(Seq.empty)
879-
924+
private def fetchRemoteManagedBuffer(
925+
blockId: BlockId,
926+
blockSize: Long,
927+
locationsAndStatus: BlockManagerMessages.BlockLocationsAndStatus): Option[ManagedBuffer] = {
880928
// If the block size is above the threshold, we should pass our FileManger to
881929
// BlockTransferService, which will leverage it to spill the block; if not, then passed-in
882930
// null value means the block will be persisted in memory.
@@ -885,8 +933,9 @@ private[spark] class BlockManager(
885933
} else {
886934
null
887935
}
888-
889-
val locations = sortLocations(blockLocations)
936+
var runningFailureCount = 0
937+
var totalFailureCount = 0
938+
val locations = sortLocations(locationsAndStatus.locations)
890939
val maxFetchFailures = locations.size
891940
var locationIterator = locations.iterator
892941
while (locationIterator.hasNext) {
@@ -946,11 +995,37 @@ private[spark] class BlockManager(
946995
None
947996
}
948997

998+
/**
999+
* Reads the block from the local directories of another executor which runs on the same host.
1000+
*/
1001+
private[spark] def readDiskBlockFromSameHostExecutor(
1002+
blockId: BlockId,
1003+
localDirs: Array[String],
1004+
blockSize: Long): Option[ManagedBuffer] = {
1005+
val file = ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, blockId.name)
1006+
if (file.exists()) {
1007+
val mangedBuffer = securityManager.getIOEncryptionKey() match {
1008+
case Some(key) =>
1009+
// Encrypted blocks cannot be memory mapped; return a special object that does decryption
1010+
// and provides InputStream / FileRegion implementations for reading the data.
1011+
new EncryptedManagedBuffer(
1012+
new EncryptedBlockData(file, blockSize, conf, key))
1013+
1014+
case _ =>
1015+
val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle")
1016+
new FileSegmentManagedBuffer(transportConf, file, 0, file.length)
1017+
}
1018+
Some(mangedBuffer)
1019+
} else {
1020+
None
1021+
}
1022+
}
1023+
9491024
/**
9501025
* Get block from remote block managers as serialized bytes.
9511026
*/
9521027
def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = {
953-
getRemoteManagedBuffer(blockId).map { data =>
1028+
getRemoteBlock(blockId, (data: ManagedBuffer) => {
9541029
// SPARK-24307 undocumented "escape-hatch" in case there are any issues in converting to
9551030
// ChunkedByteBuffer, to go back to old code-path. Can be removed post Spark 2.4 if
9561031
// new path is stable.
@@ -959,7 +1034,7 @@ private[spark] class BlockManager(
9591034
} else {
9601035
ChunkedByteBuffer.fromManagedBuffer(data)
9611036
}
962-
}
1037+
})
9631038
}
9641039

9651040
/**

0 commit comments

Comments
 (0)