Skip to content

Commit 5bb11ce

Browse files
szilard-nemethbrumi1024
authored andcommitted
HADOOP-15327. Upgrade MR ShuffleHandler to use Netty4 #3259. Contributed by Szilard Nemeth.
1 parent 552ee44 commit 5bb11ce

File tree

11 files changed

+1602
-571
lines changed

11 files changed

+1602
-571
lines changed

hadoop-client-modules/hadoop-client-runtime/pom.xml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -148,6 +148,7 @@
148148
<!-- Leave javax APIs that are stable -->
149149
<!-- the jdk ships part of the javax.annotation namespace, so if we want to relocate this we'll have to care it out by class :( -->
150150
<exclude>com.google.code.findbugs:jsr305</exclude>
151+
<exclude>io.netty:*</exclude>
151152
<exclude>io.dropwizard.metrics:metrics-core</exclude>
152153
<exclude>org.eclipse.jetty:jetty-servlet</exclude>
153154
<exclude>org.eclipse.jetty:jetty-security</exclude>

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java

Lines changed: 21 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -53,14 +53,15 @@
5353

5454
import org.apache.hadoop.classification.VisibleForTesting;
5555

56-
class Fetcher<K,V> extends Thread {
56+
@VisibleForTesting
57+
public class Fetcher<K, V> extends Thread {
5758

5859
private static final Logger LOG = LoggerFactory.getLogger(Fetcher.class);
5960

60-
/** Number of ms before timing out a copy */
61+
/** Number of ms before timing out a copy. */
6162
private static final int DEFAULT_STALLED_COPY_TIMEOUT = 3 * 60 * 1000;
6263

63-
/** Basic/unit connection timeout (in milliseconds) */
64+
/** Basic/unit connection timeout (in milliseconds). */
6465
private final static int UNIT_CONNECT_TIMEOUT = 60 * 1000;
6566

6667
/* Default read timeout (in milliseconds) */
@@ -72,19 +73,21 @@ class Fetcher<K,V> extends Thread {
7273
private static final String FETCH_RETRY_AFTER_HEADER = "Retry-After";
7374

7475
protected final Reporter reporter;
75-
private enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
76+
@VisibleForTesting
77+
public enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
7678
CONNECTION, WRONG_REDUCE}
77-
78-
private final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors";
79+
80+
@VisibleForTesting
81+
public final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors";
7982
private final JobConf jobConf;
8083
private final Counters.Counter connectionErrs;
8184
private final Counters.Counter ioErrs;
8285
private final Counters.Counter wrongLengthErrs;
8386
private final Counters.Counter badIdErrs;
8487
private final Counters.Counter wrongMapErrs;
8588
private final Counters.Counter wrongReduceErrs;
86-
protected final MergeManager<K,V> merger;
87-
protected final ShuffleSchedulerImpl<K,V> scheduler;
89+
protected final MergeManager<K, V> merger;
90+
protected final ShuffleSchedulerImpl<K, V> scheduler;
8891
protected final ShuffleClientMetrics metrics;
8992
protected final ExceptionReporter exceptionReporter;
9093
protected final int id;
@@ -111,7 +114,7 @@ private enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
111114
private static SSLFactory sslFactory;
112115

113116
public Fetcher(JobConf job, TaskAttemptID reduceId,
114-
ShuffleSchedulerImpl<K,V> scheduler, MergeManager<K,V> merger,
117+
ShuffleSchedulerImpl<K, V> scheduler, MergeManager<K, V> merger,
115118
Reporter reporter, ShuffleClientMetrics metrics,
116119
ExceptionReporter exceptionReporter, SecretKey shuffleKey) {
117120
this(job, reduceId, scheduler, merger, reporter, metrics,
@@ -120,7 +123,7 @@ public Fetcher(JobConf job, TaskAttemptID reduceId,
120123

121124
@VisibleForTesting
122125
Fetcher(JobConf job, TaskAttemptID reduceId,
123-
ShuffleSchedulerImpl<K,V> scheduler, MergeManager<K,V> merger,
126+
ShuffleSchedulerImpl<K, V> scheduler, MergeManager<K, V> merger,
124127
Reporter reporter, ShuffleClientMetrics metrics,
125128
ExceptionReporter exceptionReporter, SecretKey shuffleKey,
126129
int id) {
@@ -315,9 +318,8 @@ protected void copyFromHost(MapHost host) throws IOException {
315318
return;
316319
}
317320

318-
if(LOG.isDebugEnabled()) {
319-
LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: "
320-
+ maps);
321+
if (LOG.isDebugEnabled()) {
322+
LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: " + maps);
321323
}
322324

323325
// List of maps to be fetched yet
@@ -411,8 +413,8 @@ private void openConnectionWithRetry(URL url) throws IOException {
411413
shouldWait = false;
412414
} catch (IOException e) {
413415
if (!fetchRetryEnabled) {
414-
// throw exception directly if fetch's retry is not enabled
415-
throw e;
416+
// throw exception directly if fetch's retry is not enabled
417+
throw e;
416418
}
417419
if ((Time.monotonicNow() - startTime) >= this.fetchRetryTimeout) {
418420
LOG.warn("Failed to connect to host: " + url + "after "
@@ -489,7 +491,7 @@ private TaskAttemptID[] copyMapOutput(MapHost host,
489491
DataInputStream input,
490492
Set<TaskAttemptID> remaining,
491493
boolean canRetry) throws IOException {
492-
MapOutput<K,V> mapOutput = null;
494+
MapOutput<K, V> mapOutput = null;
493495
TaskAttemptID mapId = null;
494496
long decompressedLength = -1;
495497
long compressedLength = -1;
@@ -611,7 +613,7 @@ private void checkTimeoutOrRetry(MapHost host, IOException ioe)
611613
// First time to retry.
612614
long currentTime = Time.monotonicNow();
613615
if (retryStartTime == 0) {
614-
retryStartTime = currentTime;
616+
retryStartTime = currentTime;
615617
}
616618

617619
// Retry is not timeout, let's do retry with throwing an exception.
@@ -628,7 +630,7 @@ private void checkTimeoutOrRetry(MapHost host, IOException ioe)
628630
}
629631

630632
/**
631-
* Do some basic verification on the input received -- Being defensive
633+
* Do some basic verification on the input received -- Being defensive.
632634
* @param compressedLength
633635
* @param decompressedLength
634636
* @param forReduce
@@ -695,8 +697,7 @@ private URL getMapOutputURL(MapHost host, Collection<TaskAttemptID> maps
695697
* only on the last failure. Instead of connecting with a timeout of
696698
* X, we try connecting with a timeout of x < X but multiple times.
697699
*/
698-
private void connect(URLConnection connection, int connectionTimeout)
699-
throws IOException {
700+
private void connect(URLConnection connection, int connectionTimeout) throws IOException {
700701
int unit = 0;
701702
if (connectionTimeout < 0) {
702703
throw new IOException("Invalid timeout "

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import org.apache.hadoop.io.Text;
2727
import org.apache.hadoop.io.WritableComparator;
2828
import org.apache.hadoop.mapreduce.TaskCounter;
29+
import org.apache.hadoop.mapreduce.task.reduce.Fetcher;
2930
import org.junit.After;
3031
import org.junit.Before;
3132
import org.junit.Test;
@@ -37,6 +38,7 @@
3738
import java.util.Formatter;
3839
import java.util.Iterator;
3940

41+
import static org.apache.hadoop.mapreduce.task.reduce.Fetcher.SHUFFLE_ERR_GRP_NAME;
4042
import static org.junit.Assert.assertEquals;
4143
import static org.junit.Assert.assertTrue;
4244
import static org.junit.Assert.fail;
@@ -87,6 +89,9 @@ public void testReduceFromPartialMem() throws Exception {
8789
final long spill = c.findCounter(TaskCounter.SPILLED_RECORDS).getCounter();
8890
assertTrue("Expected some records not spilled during reduce" + spill + ")",
8991
spill < 2 * out); // spilled map records, some records at the reduce
92+
long shuffleIoErrors =
93+
c.getGroup(SHUFFLE_ERR_GRP_NAME).getCounter(Fetcher.ShuffleErrors.IO_ERROR.toString());
94+
assertEquals(0, shuffleIoErrors);
9095
}
9196

9297
/**

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java

Lines changed: 9 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,9 @@
2323
import java.io.RandomAccessFile;
2424

2525
import org.apache.hadoop.classification.VisibleForTesting;
26+
import io.netty.buffer.ByteBuf;
27+
import io.netty.buffer.ByteBufAllocator;
28+
import io.netty.handler.stream.ChunkedFile;
2629
import org.apache.hadoop.io.ReadaheadPool;
2730
import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
2831
import org.apache.hadoop.io.nativeio.NativeIO;
@@ -31,8 +34,6 @@
3134

3235
import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED;
3336

34-
import org.jboss.netty.handler.stream.ChunkedFile;
35-
3637
public class FadvisedChunkedFile extends ChunkedFile {
3738

3839
private static final Logger LOG =
@@ -64,16 +65,16 @@ FileDescriptor getFd() {
6465
}
6566

6667
@Override
67-
public Object nextChunk() throws Exception {
68+
public ByteBuf readChunk(ByteBufAllocator allocator) throws Exception {
6869
synchronized (closeLock) {
6970
if (fd.valid()) {
7071
if (manageOsCache && readaheadPool != null) {
7172
readaheadRequest = readaheadPool
7273
.readaheadStream(
73-
identifier, fd, getCurrentOffset(), readaheadLength,
74-
getEndOffset(), readaheadRequest);
74+
identifier, fd, currentOffset(), readaheadLength,
75+
endOffset(), readaheadRequest);
7576
}
76-
return super.nextChunk();
77+
return super.readChunk(allocator);
7778
} else {
7879
return null;
7980
}
@@ -88,12 +89,12 @@ public void close() throws Exception {
8889
readaheadRequest = null;
8990
}
9091
if (fd.valid() &&
91-
manageOsCache && getEndOffset() - getStartOffset() > 0) {
92+
manageOsCache && endOffset() - startOffset() > 0) {
9293
try {
9394
NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
9495
identifier,
9596
fd,
96-
getStartOffset(), getEndOffset() - getStartOffset(),
97+
startOffset(), endOffset() - startOffset(),
9798
POSIX_FADV_DONTNEED);
9899
} catch (Throwable t) {
99100
LOG.warn("Failed to manage OS cache for " + identifier +

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java

Lines changed: 7 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import java.nio.channels.FileChannel;
2626
import java.nio.channels.WritableByteChannel;
2727

28+
import io.netty.channel.DefaultFileRegion;
2829
import org.apache.hadoop.io.ReadaheadPool;
2930
import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
3031
import org.apache.hadoop.io.nativeio.NativeIO;
@@ -33,8 +34,6 @@
3334

3435
import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED;
3536

36-
import org.jboss.netty.channel.DefaultFileRegion;
37-
3837
import org.apache.hadoop.classification.VisibleForTesting;
3938

4039
public class FadvisedFileRegion extends DefaultFileRegion {
@@ -77,8 +76,8 @@ public long transferTo(WritableByteChannel target, long position)
7776
throws IOException {
7877
if (readaheadPool != null && readaheadLength > 0) {
7978
readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
80-
getPosition() + position, readaheadLength,
81-
getPosition() + getCount(), readaheadRequest);
79+
position() + position, readaheadLength,
80+
position() + count(), readaheadRequest);
8281
}
8382

8483
if(this.shuffleTransferToAllowed) {
@@ -147,22 +146,22 @@ long customShuffleTransfer(WritableByteChannel target, long position)
147146

148147

149148
@Override
150-
public void releaseExternalResources() {
149+
protected void deallocate() {
151150
if (readaheadRequest != null) {
152151
readaheadRequest.cancel();
153152
}
154-
super.releaseExternalResources();
153+
super.deallocate();
155154
}
156155

157156
/**
158157
* Call when the transfer completes successfully so we can advise the OS that
159158
* we don't need the region to be cached anymore.
160159
*/
161160
public void transferSuccessful() {
162-
if (manageOsCache && getCount() > 0) {
161+
if (manageOsCache && count() > 0) {
163162
try {
164163
NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
165-
fd, getPosition(), getCount(), POSIX_FADV_DONTNEED);
164+
fd, position(), count(), POSIX_FADV_DONTNEED);
166165
} catch (Throwable t) {
167166
LOG.warn("Failed to manage OS cache for " + identifier, t);
168167
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,106 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package org.apache.hadoop.mapred;
18+
19+
import io.netty.buffer.ByteBuf;
20+
import io.netty.channel.ChannelHandlerContext;
21+
import io.netty.channel.ChannelPromise;
22+
import io.netty.handler.codec.http.HttpHeaders;
23+
import io.netty.handler.codec.http.HttpResponse;
24+
import io.netty.handler.codec.http.HttpResponseEncoder;
25+
import org.slf4j.Logger;
26+
import org.slf4j.LoggerFactory;
27+
28+
import java.util.List;
29+
30+
class LoggingHttpResponseEncoder extends HttpResponseEncoder {
31+
private static final Logger LOG = LoggerFactory.getLogger(LoggingHttpResponseEncoder.class);
32+
private final boolean logStacktraceOfEncodingMethods;
33+
34+
LoggingHttpResponseEncoder(boolean logStacktraceOfEncodingMethods) {
35+
this.logStacktraceOfEncodingMethods = logStacktraceOfEncodingMethods;
36+
}
37+
38+
@Override
39+
public boolean acceptOutboundMessage(Object msg) throws Exception {
40+
printExecutingMethod();
41+
LOG.info("OUTBOUND MESSAGE: " + msg);
42+
return super.acceptOutboundMessage(msg);
43+
}
44+
45+
@Override
46+
protected void encodeInitialLine(ByteBuf buf, HttpResponse response) throws Exception {
47+
LOG.debug("Executing method: {}, response: {}",
48+
getExecutingMethodName(), response);
49+
logStacktraceIfRequired();
50+
super.encodeInitialLine(buf, response);
51+
}
52+
53+
@Override
54+
protected void encode(ChannelHandlerContext ctx, Object msg,
55+
List<Object> out) throws Exception {
56+
LOG.debug("Encoding to channel {}: {}", ctx.channel(), msg);
57+
printExecutingMethod();
58+
logStacktraceIfRequired();
59+
super.encode(ctx, msg, out);
60+
}
61+
62+
@Override
63+
protected void encodeHeaders(HttpHeaders headers, ByteBuf buf) {
64+
printExecutingMethod();
65+
super.encodeHeaders(headers, buf);
66+
}
67+
68+
@Override
69+
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise
70+
promise) throws Exception {
71+
LOG.debug("Writing to channel {}: {}", ctx.channel(), msg);
72+
printExecutingMethod();
73+
super.write(ctx, msg, promise);
74+
}
75+
76+
private void logStacktraceIfRequired() {
77+
if (logStacktraceOfEncodingMethods) {
78+
LOG.debug("Stacktrace: ", new Throwable());
79+
}
80+
}
81+
82+
private void printExecutingMethod() {
83+
String methodName = getExecutingMethodName(1);
84+
LOG.debug("Executing method: {}", methodName);
85+
}
86+
87+
private String getExecutingMethodName() {
88+
return getExecutingMethodName(0);
89+
}
90+
91+
private String getExecutingMethodName(int additionalSkipFrames) {
92+
try {
93+
StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace();
94+
// Array items (indices):
95+
// 0: java.lang.Thread.getStackTrace(...)
96+
// 1: TestShuffleHandler$LoggingHttpResponseEncoder.getExecutingMethodName(...)
97+
int skipFrames = 2 + additionalSkipFrames;
98+
String methodName = stackTrace[skipFrames].getMethodName();
99+
String className = this.getClass().getSimpleName();
100+
return className + "#" + methodName;
101+
} catch (Throwable t) {
102+
LOG.error("Error while getting execution method name", t);
103+
return "unknown";
104+
}
105+
}
106+
}

0 commit comments

Comments
 (0)