|
@@ -37,6 +37,7 @@ import java.io.RandomAccessFile;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.URL;
|
|
import java.net.URL;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
|
|
+import java.nio.channels.ClosedChannelException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Collections;
|
|
import java.util.Collections;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
@@ -45,6 +46,7 @@ import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.Executors;
|
|
import java.util.concurrent.Executors;
|
|
import java.util.concurrent.ThreadFactory;
|
|
import java.util.concurrent.ThreadFactory;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
+import java.util.regex.Pattern;
|
|
|
|
|
|
import javax.crypto.SecretKey;
|
|
import javax.crypto.SecretKey;
|
|
|
|
|
|
@@ -120,10 +122,16 @@ public class ShuffleHandler extends AbstractService
|
|
public static final String SHUFFLE_READAHEAD_BYTES = "mapreduce.shuffle.readahead.bytes";
|
|
public static final String SHUFFLE_READAHEAD_BYTES = "mapreduce.shuffle.readahead.bytes";
|
|
public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
|
|
public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
|
|
|
|
|
|
|
|
+ // pattern to identify errors related to the client closing the socket early
|
|
|
|
+ // idea borrowed from Netty SslHandler
|
|
|
|
+ private static final Pattern IGNORABLE_ERROR_MESSAGE = Pattern.compile(
|
|
|
|
+ "^.*(?:connection.*reset|connection.*closed|broken.*pipe).*$",
|
|
|
|
+ Pattern.CASE_INSENSITIVE);
|
|
|
|
+
|
|
private int port;
|
|
private int port;
|
|
private ChannelFactory selector;
|
|
private ChannelFactory selector;
|
|
private final ChannelGroup accepted = new DefaultChannelGroup();
|
|
private final ChannelGroup accepted = new DefaultChannelGroup();
|
|
- private HttpPipelineFactory pipelineFact;
|
|
|
|
|
|
+ protected HttpPipelineFactory pipelineFact;
|
|
private int sslFileBufferSize;
|
|
private int sslFileBufferSize;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -319,13 +327,17 @@ public class ShuffleHandler extends AbstractService
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ protected Shuffle getShuffle(Configuration conf) {
|
|
|
|
+ return new Shuffle(conf);
|
|
|
|
+ }
|
|
|
|
+
|
|
class HttpPipelineFactory implements ChannelPipelineFactory {
|
|
class HttpPipelineFactory implements ChannelPipelineFactory {
|
|
|
|
|
|
final Shuffle SHUFFLE;
|
|
final Shuffle SHUFFLE;
|
|
private SSLFactory sslFactory;
|
|
private SSLFactory sslFactory;
|
|
|
|
|
|
public HttpPipelineFactory(Configuration conf) throws Exception {
|
|
public HttpPipelineFactory(Configuration conf) throws Exception {
|
|
- SHUFFLE = new Shuffle(conf);
|
|
|
|
|
|
+ SHUFFLE = getShuffle(conf);
|
|
if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY,
|
|
if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY,
|
|
MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) {
|
|
MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) {
|
|
sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
|
|
sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
|
|
@@ -465,7 +477,7 @@ public class ShuffleHandler extends AbstractService
|
|
lastMap.addListener(ChannelFutureListener.CLOSE);
|
|
lastMap.addListener(ChannelFutureListener.CLOSE);
|
|
}
|
|
}
|
|
|
|
|
|
- private void verifyRequest(String appid, ChannelHandlerContext ctx,
|
|
|
|
|
|
+ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
|
|
HttpRequest request, HttpResponse response, URL requestUri)
|
|
HttpRequest request, HttpResponse response, URL requestUri)
|
|
throws IOException {
|
|
throws IOException {
|
|
SecretKey tokenSecret = secretManager.retrieveTokenSecret(appid);
|
|
SecretKey tokenSecret = secretManager.retrieveTokenSecret(appid);
|
|
@@ -566,12 +578,12 @@ public class ShuffleHandler extends AbstractService
|
|
return writeFuture;
|
|
return writeFuture;
|
|
}
|
|
}
|
|
|
|
|
|
- private void sendError(ChannelHandlerContext ctx,
|
|
|
|
|
|
+ protected void sendError(ChannelHandlerContext ctx,
|
|
HttpResponseStatus status) {
|
|
HttpResponseStatus status) {
|
|
sendError(ctx, "", status);
|
|
sendError(ctx, "", status);
|
|
}
|
|
}
|
|
|
|
|
|
- private void sendError(ChannelHandlerContext ctx, String message,
|
|
|
|
|
|
+ protected void sendError(ChannelHandlerContext ctx, String message,
|
|
HttpResponseStatus status) {
|
|
HttpResponseStatus status) {
|
|
HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
|
|
HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
|
|
response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
|
|
response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
|
|
@@ -590,6 +602,16 @@ public class ShuffleHandler extends AbstractService
|
|
if (cause instanceof TooLongFrameException) {
|
|
if (cause instanceof TooLongFrameException) {
|
|
sendError(ctx, BAD_REQUEST);
|
|
sendError(ctx, BAD_REQUEST);
|
|
return;
|
|
return;
|
|
|
|
+ } else if (cause instanceof IOException) {
|
|
|
|
+ if (cause instanceof ClosedChannelException) {
|
|
|
|
+ LOG.debug("Ignoring closed channel error", cause);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ String message = String.valueOf(cause.getMessage());
|
|
|
|
+ if (IGNORABLE_ERROR_MESSAGE.matcher(message).matches()) {
|
|
|
|
+ LOG.debug("Ignoring client socket close", cause);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
LOG.error("Shuffle error: ", cause);
|
|
LOG.error("Shuffle error: ", cause);
|