Skip to content

[SPARK-25299] V1 Version #10

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

Open
wants to merge 4 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.StreamCallbackWithID;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.server.OneForOneStreamManager;
import org.apache.spark.network.server.RpcHandler;
Expand Down Expand Up @@ -106,7 +107,7 @@ protected void handleMessage(

} else if (msgObj instanceof RegisterExecutor) {
final Timer.Context responseDelayContext =
metrics.registerExecutorRequestLatencyMillis.time();
metrics.registerExecutorRequestLatencyMillis.time();
try {
RegisterExecutor msg = (RegisterExecutor) msgObj;
checkAuth(client, msg.appId);
Expand All @@ -116,9 +117,49 @@ protected void handleMessage(
responseDelayContext.stop();
}

} else if (msgObj instanceof RegisterExecutorForBackupsOnly) {
final Timer.Context responseDelayContext =
metrics.registerExecutorRequestLatencyMillis.time();
try {
RegisterExecutorForBackupsOnly msg = (RegisterExecutorForBackupsOnly) msgObj;
checkAuth(client, msg.appId);
blockManager.registerExecutorForBackups(msg.appId, msg.execId, msg.shuffleManager);
callback.onSuccess(ByteBuffer.wrap(new byte[0]));
} finally {
responseDelayContext.stop();
}

} else {
throw new UnsupportedOperationException("Unexpected message: " + msgObj);
}

}

@Override
public StreamCallbackWithID receiveStream(
TransportClient client,
ByteBuffer messageHeader,
RpcResponseCallback callback) {
BlockTransferMessage header = BlockTransferMessage.Decoder.fromByteBuffer(messageHeader);
if (header instanceof UploadShuffleFileStream) {
UploadShuffleFileStream msg = (UploadShuffleFileStream) header;
checkAuth(client, msg.appId);
return blockManager.openShuffleFileForBackup(
msg.appId,
msg.execId,
msg.shuffleId,
msg.mapId);
} else if (header instanceof UploadShuffleIndexFileStream) {
UploadShuffleIndexFileStream msg = (UploadShuffleIndexFileStream) header;
checkAuth(client, msg.appId);
return blockManager.openShuffleIndexFileForBackup(
msg.appId,
msg.execId,
msg.shuffleId,
msg.mapId);
} else {
throw new UnsupportedOperationException("Unexpected message header: " + header);
}
}

public MetricSet getAllMetrics() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,12 @@
package org.apache.spark.network.shuffle;

import java.io.*;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.*;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
Expand All @@ -44,6 +49,7 @@

import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.client.StreamCallbackWithID;
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
import org.apache.spark.network.util.LevelDBProvider;
import org.apache.spark.network.util.LevelDBProvider.StoreVersion;
Expand Down Expand Up @@ -75,6 +81,8 @@ public class ExternalShuffleBlockResolver {
@VisibleForTesting
final ConcurrentMap<AppExecId, ExecutorShuffleInfo> executors;

private final ConcurrentMap<AppExecId, ExecutorShuffleInfo> backupExecutors;

/**
* Caches index file information so that we can avoid open/close the index files
* for each block fetch.
Expand All @@ -95,6 +103,7 @@ public class ExternalShuffleBlockResolver {
"org.apache.spark.shuffle.sort.SortShuffleManager",
"org.apache.spark.shuffle.unsafe.UnsafeShuffleManager");


public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile)
throws IOException {
this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor(
Expand Down Expand Up @@ -144,6 +153,12 @@ public void registerExecutor(
String execId,
ExecutorShuffleInfo executorInfo) {
AppExecId fullId = new AppExecId(appId, execId);
if (backupExecutors.containsKey(fullId)) {
throw new UnsupportedOperationException(
String.format(
"Executor %s cannot be registered for both primary shuffle management and backup" +
" shuffle management.", fullId));
}
logger.info("Registered executor {} with {}", fullId, executorInfo);
if (!knownManagers.contains(executorInfo.shuffleManager)) {
throw new UnsupportedOperationException(
Expand All @@ -161,6 +176,32 @@ public void registerExecutor(
executors.put(fullId, executorInfo);
}

private StreamCallbackWithID getFileWriterStreamCallback(
String appId,
String execId,
int shuffleId,
int mapId,
String extension,
FileWriterStreamCallback.BackupFileType backupFileType) {
AppExecId fullId = new AppExecId(appId, execId);
ExecutorShuffleInfo executor = backupExecutors.get(fullId);
if (executor == null) {
throw new RuntimeException(
String.format("Executor is not registered for shuffle file backups" +
" (appId=%s, execId=%s)", appId, execId));
}
File backedUpFile = getFile(executor.localDirs, executor.subDirsPerLocalDir,
"shuffle_" + shuffleId + "_" + mapId + "_0." + extension);
FileWriterStreamCallback streamCallback = new FileWriterStreamCallback(
fullId,
shuffleId,
mapId,
backedUpFile,
backupFileType);
streamCallback.open();
return streamCallback;
}

/**
* Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). We make assumptions
* about how the hash and sort based shuffles store their data.
Expand All @@ -173,6 +214,13 @@ public ManagedBuffer getBlockData(
int reduceId) {
ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId));
if (executor == null) {
logger.info("application's shuffle data isn't in main file system, checking backups..." +
"app id: {}, executor id: {}, shuffle id: {}, map id: {}, reduce id: {}",
appId, execId, shuffleId, mapId, reduceId);
executor = backupExecutors.get(new AppExecId(appId, execId));
}
if (executor == null) {
logger.warn("Executor is not registered (appId: {}, execId: {}", appId, execId);
throw new RuntimeException(
String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,9 @@
import org.apache.spark.network.sasl.SecretKeyHolder;
import org.apache.spark.network.server.NoOpRpcHandler;
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
import org.apache.spark.network.shuffle.protocol.RegisterDriver;
import org.apache.spark.network.shuffle.protocol.RegisterExecutor;
import org.apache.spark.network.shuffle.protocol.RegisterExecutorForBackupsOnly;
import org.apache.spark.network.util.TransportConf;

/**
Expand All @@ -43,7 +45,7 @@
* BlockTransferService), which has the downside of losing the shuffle data if we lose the
* executors.
*/
public class ExternalShuffleClient extends ShuffleClient {
public class ExternalShuffleClient extends ShuffleClient{
private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleClient.class);

private final TransportConf conf;
Expand Down Expand Up @@ -90,6 +92,7 @@ public void fetchBlocks(
int port,
String execId,
String[] blockIds,
boolean isRemote,
BlockFetchingListener listener,
DownloadFileManager downloadFileManager) {
checkInit();
Expand Down Expand Up @@ -145,6 +148,20 @@ public void registerWithShuffleServer(
}
}

public void registerWithRemoteShuffleServer(
String driverHostPort,
String host,
int port,
String execId,
String shuffleManager) throws IOException, InterruptedException{
checkInit();
try (TransportClient client = clientFactory.createUnmanagedClient(host, port)) {
ByteBuffer registerMessage = new RegisterExecutorForBackupsOnly(
driverHostPort, appId, execId, shuffleManager).toByteBuffer();
client.sendRpcSync(registerMessage, registrationTimeoutMs);
}
}

@Override
public void close() {
checkInit();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
package org.apache.spark.network.shuffle;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;

import org.apache.spark.network.client.StreamCallbackWithID;

final class FileWriterStreamCallback implements StreamCallbackWithID {

private static final Logger logger = LoggerFactory.getLogger(FileWriterStreamCallback.class);

public enum BackupFileType {
DATA("shuffle-data"),
INDEX("shuffle-index");

private final String typeString;

BackupFileType(String typeString) {
this.typeString = typeString;
}

@Override
public String toString() {
return typeString;
}
}
private final ExternalShuffleBlockResolver.AppExecId fullExecId;
private final int shuffleId;
private final int mapId;
private final File file;
private final BackupFileType fileType;
private WritableByteChannel fileOutputChannel = null;

FileWriterStreamCallback(
ExternalShuffleBlockResolver.AppExecId fullExecId,
int shuffleId,
int mapId,
File file,
BackupFileType fileType) {
this.fullExecId = fullExecId;
this.shuffleId = shuffleId;
this.mapId = mapId;
this.file = file;
this.fileType = fileType;
}

public void open() {
logger.info(
"Opening {} for backup writing. File type: {}", file.getAbsolutePath(), fileType);
if (fileOutputChannel != null) {
throw new IllegalStateException(
String.format(
"File %s for is already open for writing (type: %s).",
file.getAbsolutePath(),
fileType));
}
if (!file.exists()) {
try {
if (!file.getParentFile().isDirectory() && !file.getParentFile().mkdirs()) {
throw new IOException(
String.format(
"Failed to create shuffle file directory at"
+ file.getParentFile().getAbsolutePath() + "(type: %s).", fileType));
}

if (!file.createNewFile()) {
throw new IOException(
String.format(
"Failed to create shuffle file (type: %s).", fileType));
}
} catch (IOException e) {
throw new RuntimeException(
String.format(
"Failed to create shuffle file at %s for backup (type: %s).",
file.getAbsolutePath(),
fileType),
e);
}
}
try {
// TODO encryption
fileOutputChannel = Channels.newChannel(new FileOutputStream(file));
} catch (FileNotFoundException e) {
throw new RuntimeException(
String.format(
"Failed to find file for writing at %s (type: %s).",
file.getAbsolutePath(),
fileType),
e);
}
}

@Override
public String getID() {
return String.format("%s-%s-%d-%d-%s",
fullExecId.appId,
fullExecId.execId,
shuffleId,
mapId,
fileType);
}

@Override
public void onData(String streamId, ByteBuffer buf) throws IOException {
verifyShuffleFileOpenForWriting();
while (buf.hasRemaining()) {
fileOutputChannel.write(buf);
}
}

@Override
public void onComplete(String streamId) throws IOException {
fileOutputChannel.close();
}

@Override
public void onFailure(String streamId, Throwable cause) throws IOException {
logger.warn("Failed to back up shuffle file at {} (type: %s).",
file.getAbsolutePath(),
fileType,
cause);
fileOutputChannel.close();
// TODO delete parent dirs too
if (!file.delete()) {
logger.warn(
"Failed to delete incomplete backup shuffle file at %s (type: %s)",
file.getAbsolutePath(),
fileType);
}
}

private void verifyShuffleFileOpenForWriting() {
if (fileOutputChannel == null) {
throw new RuntimeException(
String.format(
"Shuffle file at %s not open for writing (type: %s).",
file.getAbsolutePath(),
fileType));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ public abstract void fetchBlocks(
int port,
String execId,
String[] blockIds,
boolean isRemote,
BlockFetchingListener listener,
DownloadFileManager downloadFileManager);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,15 @@
import java.util.concurrent.TimeUnit;

import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.spark.network.shuffle.protocol.mesos.ShuffleServiceHeartbeat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.sasl.SecretKeyHolder;
import org.apache.spark.network.shuffle.ExternalShuffleClient;
import org.apache.spark.network.shuffle.protocol.mesos.RegisterDriver;
import org.apache.spark.network.shuffle.protocol.RegisterDriver;
import org.apache.spark.network.shuffle.protocol.ExternalServiceHeartbeat;
import org.apache.spark.network.util.TransportConf;

/**
Expand Down Expand Up @@ -117,7 +117,7 @@ private Heartbeater(TransportClient client) {
@Override
public void run() {
// TODO: Stop sending heartbeats if the shuffle service has lost the app due to timeout
client.send(new ShuffleServiceHeartbeat(appId).toByteBuffer());
client.send(new ExternalServiceHeartbeat(appId).toByteBuffer());
}
}
}
Loading