-
Notifications
You must be signed in to change notification settings - Fork 31
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
1 parent
f582715
commit c04e36b
Showing
8 changed files
with
507 additions
and
17 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
70 changes: 70 additions & 0 deletions
70
...in/java/org/apache/spark/shuffle/ucx/reducer/compat/spark_2_1/OnOffsetsFetchCallback.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,70 @@ | ||
/* | ||
* Copyright (C) Mellanox Technologies Ltd. 2019. ALL RIGHTS RESERVED. | ||
* See file LICENSE for terms. | ||
*/ | ||
package org.apache.spark.shuffle.ucx.reducer.compat.spark_2_1; | ||
|
||
import org.apache.spark.network.shuffle.BlockFetchingListener; | ||
import org.apache.spark.shuffle.ucx.UnsafeUtils; | ||
import org.apache.spark.shuffle.ucx.memory.RegisteredMemory; | ||
import org.apache.spark.shuffle.ucx.reducer.OnBlocksFetchCallback; | ||
import org.apache.spark.shuffle.ucx.reducer.ReducerCallback; | ||
import org.apache.spark.storage.ShuffleBlockId; | ||
import org.openucx.jucx.UcxUtils; | ||
import org.openucx.jucx.ucp.UcpEndpoint; | ||
import org.openucx.jucx.ucp.UcpRemoteKey; | ||
import org.openucx.jucx.ucp.UcpRequest; | ||
|
||
import java.nio.ByteBuffer; | ||
import java.util.Map; | ||
|
||
/** | ||
* Callback, called when got all offsets for blocks | ||
*/ | ||
public class OnOffsetsFetchCallback extends ReducerCallback { | ||
private final RegisteredMemory offsetMemory; | ||
private final long[] dataAddresses; | ||
private Map<Integer, UcpRemoteKey> dataRkeysCache; | ||
|
||
public OnOffsetsFetchCallback(ShuffleBlockId[] blockIds, UcpEndpoint endpoint, BlockFetchingListener listener, | ||
RegisteredMemory offsetMemory, long[] dataAddresses, | ||
Map<Integer, UcpRemoteKey> dataRkeysCache) { | ||
super(blockIds, endpoint, listener); | ||
this.offsetMemory = offsetMemory; | ||
this.dataAddresses = dataAddresses; | ||
this.dataRkeysCache = dataRkeysCache; | ||
} | ||
|
||
@Override | ||
public void onSuccess(UcpRequest request) { | ||
ByteBuffer resultOffset = offsetMemory.getBuffer(); | ||
long totalSize = 0; | ||
int[] sizes = new int[blockIds.length]; | ||
int offsetSize = UnsafeUtils.LONG_SIZE; | ||
for (int i = 0; i < blockIds.length; i++) { | ||
// Blocks in metadata buffer are in form | blockOffsetStart | blockOffsetEnd | | ||
long blockOffset = resultOffset.getLong(i * 2 * offsetSize); | ||
long blockLength = resultOffset.getLong(i * 2 * offsetSize + offsetSize) - blockOffset; | ||
assert (blockLength > 0) && (blockLength <= Integer.MAX_VALUE); | ||
sizes[i] = (int) blockLength; | ||
totalSize += blockLength; | ||
dataAddresses[i] += blockOffset; | ||
} | ||
|
||
assert (totalSize > 0) && (totalSize < Integer.MAX_VALUE); | ||
mempool.put(offsetMemory); | ||
RegisteredMemory blocksMemory = mempool.get((int) totalSize); | ||
|
||
long offset = 0; | ||
// Submits N fetch blocks requests | ||
for (int i = 0; i < blockIds.length; i++) { | ||
endpoint.getNonBlockingImplicit(dataAddresses[i], dataRkeysCache.get(((ShuffleBlockId)blockIds[i]).mapId()), | ||
UcxUtils.getAddress(blocksMemory.getBuffer()) + offset, sizes[i]); | ||
offset += sizes[i]; | ||
} | ||
|
||
// Process blocks when all fetched. | ||
// Flush guarantees that callback would invoke when all fetch requests will completed. | ||
endpoint.flushNonBlocking(new OnBlocksFetchCallback(this, blocksMemory, sizes)); | ||
} | ||
} |
111 changes: 111 additions & 0 deletions
111
src/main/java/org/apache/spark/shuffle/ucx/reducer/compat/spark_2_1/UcxShuffleClient.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,111 @@ | ||
/* | ||
* Copyright (C) Mellanox Technologies Ltd. 2019. ALL RIGHTS RESERVED. | ||
* See file LICENSE for terms. | ||
*/ | ||
package org.apache.spark.shuffle.ucx.reducer.compat.spark_2_1; | ||
|
||
import org.apache.spark.SparkEnv; | ||
import org.apache.spark.executor.TempShuffleReadMetrics; | ||
import org.apache.spark.network.shuffle.BlockFetchingListener; | ||
import org.apache.spark.network.shuffle.ShuffleClient; | ||
import org.apache.spark.shuffle.DriverMetadata; | ||
import org.apache.spark.shuffle.UcxShuffleManager; | ||
import org.apache.spark.shuffle.UcxWorkerWrapper; | ||
import org.apache.spark.shuffle.ucx.UnsafeUtils; | ||
import org.apache.spark.shuffle.ucx.memory.MemoryPool; | ||
import org.apache.spark.shuffle.ucx.memory.RegisteredMemory; | ||
import org.apache.spark.storage.BlockId; | ||
import org.apache.spark.storage.BlockManagerId; | ||
import org.apache.spark.storage.ShuffleBlockId; | ||
import org.openucx.jucx.UcxUtils; | ||
import org.openucx.jucx.ucp.UcpEndpoint; | ||
import org.openucx.jucx.ucp.UcpRemoteKey; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
import scala.Option; | ||
|
||
import java.util.Arrays; | ||
import java.util.HashMap; | ||
|
||
public class UcxShuffleClient extends ShuffleClient { | ||
private final MemoryPool mempool; | ||
private static final Logger logger = LoggerFactory.getLogger(UcxShuffleClient.class); | ||
private final UcxShuffleManager ucxShuffleManager; | ||
private final TempShuffleReadMetrics shuffleReadMetrics; | ||
private final UcxWorkerWrapper workerWrapper; | ||
final HashMap<Integer, UcpRemoteKey> offsetRkeysCache = new HashMap<>(); | ||
final HashMap<Integer, UcpRemoteKey> dataRkeysCache = new HashMap<>(); | ||
|
||
public UcxShuffleClient(TempShuffleReadMetrics shuffleReadMetrics, | ||
UcxWorkerWrapper workerWrapper) { | ||
this.ucxShuffleManager = (UcxShuffleManager) SparkEnv.get().shuffleManager(); | ||
this.mempool = ucxShuffleManager.ucxNode().getMemoryPool(); | ||
this.shuffleReadMetrics = shuffleReadMetrics; | ||
this.workerWrapper = workerWrapper; | ||
} | ||
|
||
/** | ||
* Submits n non blocking fetch offsets to get needed offsets for n blocks. | ||
*/ | ||
private void submitFetchOffsets(UcpEndpoint endpoint, ShuffleBlockId[] blockIds, | ||
long[] dataAddresses, RegisteredMemory offsetMemory) { | ||
DriverMetadata driverMetadata = workerWrapper.fetchDriverMetadataBuffer(blockIds[0].shuffleId()); | ||
for (int i = 0; i < blockIds.length; i++) { | ||
ShuffleBlockId blockId = blockIds[i]; | ||
|
||
long offsetAddress = driverMetadata.offsetAddress(blockId.mapId()); | ||
dataAddresses[i] = driverMetadata.dataAddress(blockId.mapId()); | ||
|
||
offsetRkeysCache.computeIfAbsent(blockId.mapId(), mapId -> | ||
endpoint.unpackRemoteKey(driverMetadata.offsetRkey(blockId.mapId()))); | ||
|
||
dataRkeysCache.computeIfAbsent(blockId.mapId(), mapId -> | ||
endpoint.unpackRemoteKey(driverMetadata.dataRkey(blockId.mapId()))); | ||
|
||
endpoint.getNonBlockingImplicit( | ||
offsetAddress + blockId.reduceId() * UnsafeUtils.LONG_SIZE, | ||
offsetRkeysCache.get(blockId.mapId()), | ||
UcxUtils.getAddress(offsetMemory.getBuffer()) + (i * 2L * UnsafeUtils.LONG_SIZE), | ||
2L * UnsafeUtils.LONG_SIZE); | ||
} | ||
} | ||
|
||
/** | ||
* Reducer entry point. Fetches remote blocks, using 2 ucp_get calls. | ||
* This method is inside ShuffleFetchIterator's for loop over hosts. | ||
* First fetches block offset from index file, and then fetches block itself. | ||
*/ | ||
@Override | ||
public void fetchBlocks(String host, int port, String execId, | ||
String[] blockIds, BlockFetchingListener listener) { | ||
long startTime = System.currentTimeMillis(); | ||
|
||
BlockManagerId blockManagerId = BlockManagerId.apply(execId, host, port, Option.empty()); | ||
UcpEndpoint endpoint = workerWrapper.getConnection(blockManagerId); | ||
|
||
long[] dataAddresses = new long[blockIds.length]; | ||
|
||
// Need to fetch 2 long offsets current block + next block to calculate exact block size. | ||
RegisteredMemory offsetMemory = mempool.get(2 * UnsafeUtils.LONG_SIZE * blockIds.length); | ||
|
||
ShuffleBlockId[] shuffleBlockIds = Arrays.stream(blockIds) | ||
.map(blockId -> (ShuffleBlockId) BlockId.apply(blockId)).toArray(ShuffleBlockId[]::new); | ||
|
||
// Submits N implicit get requests without callback | ||
submitFetchOffsets(endpoint, shuffleBlockIds, dataAddresses, offsetMemory); | ||
|
||
// flush guarantees that all that requests completes when callback is called. | ||
// TODO: fix https://github.com/openucx/ucx/issues/4267 and use endpoint flush. | ||
workerWrapper.worker().flushNonBlocking( | ||
new OnOffsetsFetchCallback(shuffleBlockIds, endpoint, listener, offsetMemory, | ||
dataAddresses, dataRkeysCache)); | ||
shuffleReadMetrics.incFetchWaitTime(System.currentTimeMillis() - startTime); | ||
} | ||
|
||
@Override | ||
public void close() { | ||
offsetRkeysCache.values().forEach(UcpRemoteKey::close); | ||
dataRkeysCache.values().forEach(UcpRemoteKey::close); | ||
logger.info("Shuffle read metrics, fetch wait time: {}ms", shuffleReadMetrics.fetchWaitTime()); | ||
} | ||
} |
44 changes: 44 additions & 0 deletions
44
src/main/scala/org/apache/spark/shuffle/compat/spark_2_1/UcxShuffleBlockResolver.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,44 @@ | ||
/* | ||
* Copyright (C) Mellanox Technologies Ltd. 2019. ALL RIGHTS RESERVED. | ||
* See file LICENSE for terms. | ||
*/ | ||
package org.apache.spark.shuffle.compat.spark_2_1 | ||
|
||
import java.io.{File, RandomAccessFile} | ||
|
||
import org.apache.spark.SparkEnv | ||
import org.apache.spark.shuffle.{CommonUcxShuffleBlockResolver, CommonUcxShuffleManager, IndexShuffleBlockResolver} | ||
import org.apache.spark.storage.ShuffleIndexBlockId | ||
|
||
/** | ||
* Mapper entry point for UcxShuffle plugin. Performs memory registration | ||
* of data and index files and publish addresses to driver metadata buffer. | ||
*/ | ||
class UcxShuffleBlockResolver(ucxShuffleManager: CommonUcxShuffleManager) | ||
extends CommonUcxShuffleBlockResolver(ucxShuffleManager) { | ||
|
||
private def getIndexFile(shuffleId: Int, mapId: Int): File = { | ||
SparkEnv.get.blockManager | ||
.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID)) | ||
} | ||
|
||
/** | ||
* Mapper commit protocol extension. Register index and data files and publish all needed | ||
* metadata to driver. | ||
*/ | ||
override def writeIndexFileAndCommit(shuffleId: ShuffleId, mapId: Int, | ||
lengths: Array[Long], dataTmp: File): Unit = { | ||
super.writeIndexFileAndCommit(shuffleId, mapId, lengths, dataTmp) | ||
val dataFile = getDataFile(shuffleId, mapId) | ||
val dataBackFile = new RandomAccessFile(dataFile, "rw") | ||
|
||
if (dataBackFile.length() == 0) { | ||
dataBackFile.close() | ||
return | ||
} | ||
|
||
val indexFile = getIndexFile(shuffleId, mapId) | ||
val indexBackFile = new RandomAccessFile(indexFile, "rw") | ||
writeIndexFileAndCommitCommon(shuffleId, mapId, lengths, dataTmp, indexBackFile, dataBackFile) | ||
} | ||
} |
Oops, something went wrong.