|
@@ -92,7 +92,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Cont
|
|
|
import org.apache.hadoop.yarn.server.records.Version;
|
|
|
import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
|
|
|
import org.apache.hadoop.yarn.server.utils.LeveldbIterator;
|
|
|
-import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.fusesource.leveldbjni.JniDBFactory;
|
|
|
import org.fusesource.leveldbjni.internal.NativeDB;
|
|
|
import org.iq80.leveldb.DB;
|
|
@@ -166,6 +165,12 @@ public class ShuffleHandler extends AuxiliaryService {
|
|
|
private static final String DATA_FILE_NAME = "file.out";
|
|
|
private static final String INDEX_FILE_NAME = "file.out.index";
|
|
|
|
|
|
+ public static final HttpResponseStatus TOO_MANY_REQ_STATUS =
|
|
|
+ new HttpResponseStatus(429, "TOO MANY REQUESTS");
|
|
|
+ // This should kept in sync with Fetcher.FETCH_RETRY_DELAY_DEFAULT
|
|
|
+ public static final long FETCH_RETRY_DELAY = 1000L;
|
|
|
+ public static final String RETRY_AFTER_HEADER = "Retry-After";
|
|
|
+
|
|
|
private int port;
|
|
|
private ChannelFactory selector;
|
|
|
private final ChannelGroup accepted = new DefaultChannelGroup();
|
|
@@ -795,7 +800,6 @@ public class ShuffleHandler extends AuxiliaryService {
|
|
|
}
|
|
|
|
|
|
class Shuffle extends SimpleChannelUpstreamHandler {
|
|
|
-
|
|
|
private static final int MAX_WEIGHT = 10 * 1024 * 1024;
|
|
|
private static final int EXPIRE_AFTER_ACCESS_MINUTES = 5;
|
|
|
private static final int ALLOWED_CONCURRENCY = 16;
|
|
@@ -875,7 +879,14 @@ public class ShuffleHandler extends AuxiliaryService {
|
|
|
LOG.info(String.format("Current number of shuffle connections (%d) is " +
|
|
|
"greater than or equal to the max allowed shuffle connections (%d)",
|
|
|
accepted.size(), maxShuffleConnections));
|
|
|
- evt.getChannel().close();
|
|
|
+
|
|
|
+ Map<String, String> headers = new HashMap<String, String>(1);
|
|
|
+ // notify fetchers to backoff for a while before closing the connection
|
|
|
+ // if the shuffle connection limit is hit. Fetchers are expected to
|
|
|
+ // handle this notification gracefully, that is, not treating this as a
|
|
|
+ // fetch failure.
|
|
|
+ headers.put(RETRY_AFTER_HEADER, String.valueOf(FETCH_RETRY_DELAY));
|
|
|
+ sendError(ctx, "", TOO_MANY_REQ_STATUS, headers);
|
|
|
return;
|
|
|
}
|
|
|
accepted.add(evt.getChannel());
|
|
@@ -1245,6 +1256,11 @@ public class ShuffleHandler extends AuxiliaryService {
|
|
|
|
|
|
protected void sendError(ChannelHandlerContext ctx, String message,
|
|
|
HttpResponseStatus status) {
|
|
|
+ sendError(ctx, message, status, Collections.<String, String>emptyMap());
|
|
|
+ }
|
|
|
+
|
|
|
+ protected void sendError(ChannelHandlerContext ctx, String msg,
|
|
|
+ HttpResponseStatus status, Map<String, String> headers) {
|
|
|
HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
|
|
|
response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8");
|
|
|
// Put shuffle version into http header
|
|
@@ -1252,8 +1268,11 @@ public class ShuffleHandler extends AuxiliaryService {
|
|
|
ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
|
|
|
response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION,
|
|
|
ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
|
|
|
+ for (Map.Entry<String, String> header : headers.entrySet()) {
|
|
|
+ response.headers().set(header.getKey(), header.getValue());
|
|
|
+ }
|
|
|
response.setContent(
|
|
|
- ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
|
|
|
+ ChannelBuffers.copiedBuffer(msg, CharsetUtil.UTF_8));
|
|
|
|
|
|
// Close the connection as soon as the error message is sent.
|
|
|
ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
|