-
Notifications
You must be signed in to change notification settings - Fork 9.1k
HADOOP-15327. Upgrade MR ShuffleHandler to use Netty4 #3259
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
Closed
Closed
Changes from all commits
Commits
Show all changes
46 commits
Select commit
Hold shift + click to select a range
dd03fd5
[WIP] This is the first version that compiles.
szilard-nemeth 89b0efd
Update
szilard-nemeth eda8f0f
ShuffleHandler: ch.isOpen() --> ch.isActive()
szilard-nemeth f657881
TestShuffleHandler: Fix mocking in testSendMapCount + replace ch.writ…
szilard-nemeth 8efba68
TestShuffleHandler.testMaxConnections: Rewrite test + production code…
szilard-nemeth af59f54
ShuffleHandler: Fix keepalive test + writing HTTP response properly t…
szilard-nemeth 7a003db
channel.closeFuture().awaitUninterruptibly() --> channel.close()
szilard-nemeth c8c6a8f
Modify TODOs
szilard-nemeth be1c67a
TestShuffleHandler: Add error handling + assertion logic
szilard-nemeth 72d6cdf
TestShuffleHandler#testSocketKeepAlive: assert HTTP response code + a…
szilard-nemeth 72db0f0
Fix idle state handling + add test
szilard-nemeth def6ef1
Add explanation: LastHttpContent.EMPTY_LAST_CONTENT
szilard-nemeth 4c6123e
Fix javac + checkstyle + whitespace issues
szilard-nemeth 9a4f963
Attempt to fix tests
szilard-nemeth 7547277
code cleanup
szilard-nemeth a1fb09c
Fix TestShuffleHandler.LoggingHttpResponseEncoder.getExecutingMethodName
szilard-nemeth 368bbdb
Turn back on outbound exception handler in tests
szilard-nemeth 99689d5
TestShuffleHandler: Introduced InputStreamReadResult that stores resp…
szilard-nemeth a25aaf3
TestShuffleHandler: Use DEFAULT_PORT for all shuffle handler port con…
szilard-nemeth 42138ce
Create class: TestExecution: Configure proxy, keep alive connection t…
szilard-nemeth 0618116
TestExecution: Configure port
szilard-nemeth 7c9dd4d
Add logging response encoder to TestShuffleHandler.testMapFileAccess
szilard-nemeth 34bd098
TestShuffleHandler.testMapFileAccess: Modify to be able to run it loc…
szilard-nemeth 23dd755
TestShuffleHandler.testMapFileAccess: Fix in production code
szilard-nemeth a25d845
TestShuffleHandler.testMapFileAccess: Add back original assertions
szilard-nemeth c579a85
Turn off debug mode
szilard-nemeth 948aece
TestShuffleHandler: Stop shufflehandler in all tests, fix debug mode …
szilard-nemeth 56680cf
TestShuffleHandler.testSocketKeepAlive: Add back assertion
szilard-nemeth 23a4c03
testReduceFromPartialMem: Add Shuffle IO error assertion to test
szilard-nemeth 9e55965
LoggingHttpResponseEncoder: Add some new logs
szilard-nemeth 0f2fb11
Fixed error handling + LastHttpContent
szilard-nemeth 35d2f04
ShuffleHandlerTest fixes + enhancements
szilard-nemeth 395f541
Fixed Gergo's comments
szilard-nemeth f396a31
Fix comments from Andras
szilard-nemeth 77c1095
Fix comments from Andras: Use Collections.emptyList() where possible …
szilard-nemeth a8efbf9
Fix comments from Andras: Fix handling the maximum number of connections
szilard-nemeth 2bd9d75
Fix comments from Andras: Replace GlobalEventExecutor.INSTANCE with a…
szilard-nemeth 6145df4
Code cleanup in TestShuffleHandler.java
szilard-nemeth 1994a19
Attempt to fix shading
szilard-nemeth 741cf52
Empty-Commit
szilard-nemeth 5c6f5b4
Empty-Commit
szilard-nemeth 1690178
Empty-Commit
szilard-nemeth b9f364d
Fix javac issues: assertThat deprecation
szilard-nemeth 1ae403b
fix blanks
szilard-nemeth 4aa8d45
Fix checkstyle
szilard-nemeth 6f36b38
fix review comments
szilard-nemeth File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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 hidden or 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 |
---|---|---|
|
@@ -53,14 +53,15 @@ | |
|
||
import org.apache.hadoop.classification.VisibleForTesting; | ||
|
||
class Fetcher<K,V> extends Thread { | ||
@VisibleForTesting | ||
public class Fetcher<K, V> extends Thread { | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(Fetcher.class); | ||
|
||
/** Number of ms before timing out a copy */ | ||
/** Number of ms before timing out a copy. */ | ||
private static final int DEFAULT_STALLED_COPY_TIMEOUT = 3 * 60 * 1000; | ||
|
||
/** Basic/unit connection timeout (in milliseconds) */ | ||
/** Basic/unit connection timeout (in milliseconds). */ | ||
private final static int UNIT_CONNECT_TIMEOUT = 60 * 1000; | ||
|
||
/* Default read timeout (in milliseconds) */ | ||
|
@@ -72,19 +73,21 @@ class Fetcher<K,V> extends Thread { | |
private static final String FETCH_RETRY_AFTER_HEADER = "Retry-After"; | ||
|
||
protected final Reporter reporter; | ||
private enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP, | ||
@VisibleForTesting | ||
public enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP, | ||
CONNECTION, WRONG_REDUCE} | ||
|
||
private final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors"; | ||
|
||
@VisibleForTesting | ||
public final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors"; | ||
private final JobConf jobConf; | ||
private final Counters.Counter connectionErrs; | ||
private final Counters.Counter ioErrs; | ||
private final Counters.Counter wrongLengthErrs; | ||
private final Counters.Counter badIdErrs; | ||
private final Counters.Counter wrongMapErrs; | ||
private final Counters.Counter wrongReduceErrs; | ||
protected final MergeManager<K,V> merger; | ||
protected final ShuffleSchedulerImpl<K,V> scheduler; | ||
protected final MergeManager<K, V> merger; | ||
protected final ShuffleSchedulerImpl<K, V> scheduler; | ||
protected final ShuffleClientMetrics metrics; | ||
protected final ExceptionReporter exceptionReporter; | ||
protected final int id; | ||
|
@@ -111,7 +114,7 @@ private enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP, | |
private static SSLFactory sslFactory; | ||
|
||
public Fetcher(JobConf job, TaskAttemptID reduceId, | ||
ShuffleSchedulerImpl<K,V> scheduler, MergeManager<K,V> merger, | ||
ShuffleSchedulerImpl<K, V> scheduler, MergeManager<K, V> merger, | ||
Reporter reporter, ShuffleClientMetrics metrics, | ||
ExceptionReporter exceptionReporter, SecretKey shuffleKey) { | ||
this(job, reduceId, scheduler, merger, reporter, metrics, | ||
|
@@ -120,7 +123,7 @@ public Fetcher(JobConf job, TaskAttemptID reduceId, | |
|
||
@VisibleForTesting | ||
Fetcher(JobConf job, TaskAttemptID reduceId, | ||
ShuffleSchedulerImpl<K,V> scheduler, MergeManager<K,V> merger, | ||
ShuffleSchedulerImpl<K, V> scheduler, MergeManager<K, V> merger, | ||
Reporter reporter, ShuffleClientMetrics metrics, | ||
ExceptionReporter exceptionReporter, SecretKey shuffleKey, | ||
int id) { | ||
|
@@ -315,9 +318,8 @@ protected void copyFromHost(MapHost host) throws IOException { | |
return; | ||
} | ||
|
||
if(LOG.isDebugEnabled()) { | ||
LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: " | ||
+ maps); | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: " + maps); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. slf4j logger messages can be rewritten using parameterized logging format. But let's not worry about that now. This PR is already too big. |
||
} | ||
|
||
// List of maps to be fetched yet | ||
|
@@ -411,8 +413,8 @@ private void openConnectionWithRetry(URL url) throws IOException { | |
shouldWait = false; | ||
} catch (IOException e) { | ||
if (!fetchRetryEnabled) { | ||
// throw exception directly if fetch's retry is not enabled | ||
throw e; | ||
// throw exception directly if fetch's retry is not enabled | ||
throw e; | ||
} | ||
if ((Time.monotonicNow() - startTime) >= this.fetchRetryTimeout) { | ||
LOG.warn("Failed to connect to host: " + url + "after " | ||
|
@@ -489,7 +491,7 @@ private TaskAttemptID[] copyMapOutput(MapHost host, | |
DataInputStream input, | ||
Set<TaskAttemptID> remaining, | ||
boolean canRetry) throws IOException { | ||
MapOutput<K,V> mapOutput = null; | ||
MapOutput<K, V> mapOutput = null; | ||
TaskAttemptID mapId = null; | ||
long decompressedLength = -1; | ||
long compressedLength = -1; | ||
|
@@ -611,7 +613,7 @@ private void checkTimeoutOrRetry(MapHost host, IOException ioe) | |
// First time to retry. | ||
long currentTime = Time.monotonicNow(); | ||
if (retryStartTime == 0) { | ||
retryStartTime = currentTime; | ||
retryStartTime = currentTime; | ||
} | ||
|
||
// Retry is not timeout, let's do retry with throwing an exception. | ||
|
@@ -628,7 +630,7 @@ private void checkTimeoutOrRetry(MapHost host, IOException ioe) | |
} | ||
|
||
/** | ||
* Do some basic verification on the input received -- Being defensive | ||
* Do some basic verification on the input received -- Being defensive. | ||
* @param compressedLength | ||
* @param decompressedLength | ||
* @param forReduce | ||
|
@@ -695,8 +697,7 @@ private URL getMapOutputURL(MapHost host, Collection<TaskAttemptID> maps | |
* only on the last failure. Instead of connecting with a timeout of | ||
* X, we try connecting with a timeout of x < X but multiple times. | ||
*/ | ||
private void connect(URLConnection connection, int connectionTimeout) | ||
throws IOException { | ||
private void connect(URLConnection connection, int connectionTimeout) throws IOException { | ||
int unit = 0; | ||
if (connectionTimeout < 0) { | ||
throw new IOException("Invalid timeout " | ||
|
This file contains hidden or 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 hidden or 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 hidden or 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
106 changes: 106 additions & 0 deletions
106
...uce-client-shuffle/src/main/java/org/apache/hadoop/mapred/LoggingHttpResponseEncoder.java
This file contains hidden or 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,106 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hadoop.mapred; | ||
|
||
import io.netty.buffer.ByteBuf; | ||
import io.netty.channel.ChannelHandlerContext; | ||
import io.netty.channel.ChannelPromise; | ||
import io.netty.handler.codec.http.HttpHeaders; | ||
import io.netty.handler.codec.http.HttpResponse; | ||
import io.netty.handler.codec.http.HttpResponseEncoder; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.util.List; | ||
|
||
class LoggingHttpResponseEncoder extends HttpResponseEncoder { | ||
private static final Logger LOG = LoggerFactory.getLogger(LoggingHttpResponseEncoder.class); | ||
private final boolean logStacktraceOfEncodingMethods; | ||
|
||
LoggingHttpResponseEncoder(boolean logStacktraceOfEncodingMethods) { | ||
this.logStacktraceOfEncodingMethods = logStacktraceOfEncodingMethods; | ||
} | ||
|
||
@Override | ||
public boolean acceptOutboundMessage(Object msg) throws Exception { | ||
printExecutingMethod(); | ||
LOG.info("OUTBOUND MESSAGE: " + msg); | ||
return super.acceptOutboundMessage(msg); | ||
} | ||
|
||
@Override | ||
protected void encodeInitialLine(ByteBuf buf, HttpResponse response) throws Exception { | ||
LOG.debug("Executing method: {}, response: {}", | ||
getExecutingMethodName(), response); | ||
logStacktraceIfRequired(); | ||
super.encodeInitialLine(buf, response); | ||
} | ||
|
||
@Override | ||
protected void encode(ChannelHandlerContext ctx, Object msg, | ||
List<Object> out) throws Exception { | ||
LOG.debug("Encoding to channel {}: {}", ctx.channel(), msg); | ||
printExecutingMethod(); | ||
logStacktraceIfRequired(); | ||
super.encode(ctx, msg, out); | ||
} | ||
|
||
@Override | ||
protected void encodeHeaders(HttpHeaders headers, ByteBuf buf) { | ||
printExecutingMethod(); | ||
super.encodeHeaders(headers, buf); | ||
} | ||
|
||
@Override | ||
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise | ||
promise) throws Exception { | ||
LOG.debug("Writing to channel {}: {}", ctx.channel(), msg); | ||
printExecutingMethod(); | ||
super.write(ctx, msg, promise); | ||
} | ||
|
||
private void logStacktraceIfRequired() { | ||
if (logStacktraceOfEncodingMethods) { | ||
LOG.debug("Stacktrace: ", new Throwable()); | ||
} | ||
} | ||
|
||
private void printExecutingMethod() { | ||
String methodName = getExecutingMethodName(1); | ||
LOG.debug("Executing method: {}", methodName); | ||
} | ||
|
||
private String getExecutingMethodName() { | ||
return getExecutingMethodName(0); | ||
} | ||
|
||
private String getExecutingMethodName(int additionalSkipFrames) { | ||
try { | ||
StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace(); | ||
// Array items (indices): | ||
// 0: java.lang.Thread.getStackTrace(...) | ||
// 1: TestShuffleHandler$LoggingHttpResponseEncoder.getExecutingMethodName(...) | ||
int skipFrames = 2 + additionalSkipFrames; | ||
String methodName = stackTrace[skipFrames].getMethodName(); | ||
String className = this.getClass().getSimpleName(); | ||
return className + "#" + methodName; | ||
} catch (Throwable t) { | ||
LOG.error("Error while getting execution method name", t); | ||
return "unknown"; | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.