Skip to content
Closed
Show file tree
Hide file tree
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 Jun 4, 2021
89b0efd
Update
szilard-nemeth Jun 4, 2021
eda8f0f
ShuffleHandler: ch.isOpen() --> ch.isActive()
szilard-nemeth Jun 7, 2021
f657881
TestShuffleHandler: Fix mocking in testSendMapCount + replace ch.writ…
szilard-nemeth Jun 7, 2021
8efba68
TestShuffleHandler.testMaxConnections: Rewrite test + production code…
szilard-nemeth Jun 8, 2021
af59f54
ShuffleHandler: Fix keepalive test + writing HTTP response properly t…
szilard-nemeth Jun 10, 2021
7a003db
channel.closeFuture().awaitUninterruptibly() --> channel.close()
szilard-nemeth Jun 10, 2021
c8c6a8f
Modify TODOs
szilard-nemeth Jun 10, 2021
be1c67a
TestShuffleHandler: Add error handling + assertion logic
szilard-nemeth Jun 11, 2021
72d6cdf
TestShuffleHandler#testSocketKeepAlive: assert HTTP response code + a…
szilard-nemeth Jun 11, 2021
72db0f0
Fix idle state handling + add test
szilard-nemeth Jun 11, 2021
def6ef1
Add explanation: LastHttpContent.EMPTY_LAST_CONTENT
szilard-nemeth Jun 11, 2021
4c6123e
Fix javac + checkstyle + whitespace issues
szilard-nemeth Jun 12, 2021
9a4f963
Attempt to fix tests
szilard-nemeth Jun 12, 2021
7547277
code cleanup
szilard-nemeth Jun 12, 2021
a1fb09c
Fix TestShuffleHandler.LoggingHttpResponseEncoder.getExecutingMethodName
szilard-nemeth Jun 15, 2021
368bbdb
Turn back on outbound exception handler in tests
szilard-nemeth Jun 15, 2021
99689d5
TestShuffleHandler: Introduced InputStreamReadResult that stores resp…
szilard-nemeth Jun 15, 2021
a25aaf3
TestShuffleHandler: Use DEFAULT_PORT for all shuffle handler port con…
szilard-nemeth Jun 15, 2021
42138ce
Create class: TestExecution: Configure proxy, keep alive connection t…
szilard-nemeth Jun 15, 2021
0618116
TestExecution: Configure port
szilard-nemeth Jun 15, 2021
7c9dd4d
Add logging response encoder to TestShuffleHandler.testMapFileAccess
szilard-nemeth Jun 15, 2021
34bd098
TestShuffleHandler.testMapFileAccess: Modify to be able to run it loc…
szilard-nemeth Jun 15, 2021
23dd755
TestShuffleHandler.testMapFileAccess: Fix in production code
szilard-nemeth Jun 15, 2021
a25d845
TestShuffleHandler.testMapFileAccess: Add back original assertions
szilard-nemeth Jun 15, 2021
c579a85
Turn off debug mode
szilard-nemeth Jun 15, 2021
948aece
TestShuffleHandler: Stop shufflehandler in all tests, fix debug mode …
szilard-nemeth Jun 15, 2021
56680cf
TestShuffleHandler.testSocketKeepAlive: Add back assertion
szilard-nemeth Jun 15, 2021
23a4c03
testReduceFromPartialMem: Add Shuffle IO error assertion to test
szilard-nemeth Jun 22, 2021
9e55965
LoggingHttpResponseEncoder: Add some new logs
szilard-nemeth Jun 28, 2021
0f2fb11
Fixed error handling + LastHttpContent
szilard-nemeth Jun 29, 2021
35d2f04
ShuffleHandlerTest fixes + enhancements
szilard-nemeth Jun 29, 2021
395f541
Fixed Gergo's comments
szilard-nemeth Aug 6, 2021
f396a31
Fix comments from Andras
szilard-nemeth Apr 6, 2022
77c1095
Fix comments from Andras: Use Collections.emptyList() where possible …
szilard-nemeth Apr 7, 2022
a8efbf9
Fix comments from Andras: Fix handling the maximum number of connections
szilard-nemeth Apr 7, 2022
2bd9d75
Fix comments from Andras: Replace GlobalEventExecutor.INSTANCE with a…
szilard-nemeth Apr 7, 2022
6145df4
Code cleanup in TestShuffleHandler.java
szilard-nemeth Apr 7, 2022
1994a19
Attempt to fix shading
szilard-nemeth Apr 11, 2022
741cf52
Empty-Commit
szilard-nemeth May 23, 2022
5c6f5b4
Empty-Commit
szilard-nemeth Jun 14, 2022
1690178
Empty-Commit
szilard-nemeth Jun 28, 2022
b9f364d
Fix javac issues: assertThat deprecation
szilard-nemeth Jun 29, 2022
1ae403b
fix blanks
szilard-nemeth Jun 29, 2022
4aa8d45
Fix checkstyle
szilard-nemeth Jun 29, 2022
6f36b38
fix review comments
szilard-nemeth Nov 10, 2022
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
1 change: 1 addition & 0 deletions hadoop-client-modules/hadoop-client-runtime/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,7 @@
<!-- Leave javax APIs that are stable -->
<!-- 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 :( -->
<exclude>com.google.code.findbugs:jsr305</exclude>
<exclude>io.netty:*</exclude>
<exclude>io.dropwizard.metrics:metrics-core</exclude>
<exclude>org.eclipse.jetty:jetty-servlet</exclude>
<exclude>org.eclipse.jetty:jetty-security</exclude>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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) */
Expand All @@ -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;
Expand All @@ -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,
Expand All @@ -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) {
Expand Down Expand Up @@ -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);
Copy link
Contributor

Choose a reason for hiding this comment

The 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
Expand Down Expand Up @@ -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 "
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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.
Expand All @@ -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
Expand Down Expand Up @@ -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 "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.task.reduce.Fetcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
Expand All @@ -37,6 +38,7 @@
import java.util.Formatter;
import java.util.Iterator;

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

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@
import java.io.RandomAccessFile;

import org.apache.hadoop.classification.VisibleForTesting;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.handler.stream.ChunkedFile;
import org.apache.hadoop.io.ReadaheadPool;
import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
import org.apache.hadoop.io.nativeio.NativeIO;
Expand All @@ -31,8 +34,6 @@

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

import org.jboss.netty.handler.stream.ChunkedFile;

public class FadvisedChunkedFile extends ChunkedFile {

private static final Logger LOG =
Expand Down Expand Up @@ -64,16 +65,16 @@ FileDescriptor getFd() {
}

@Override
public Object nextChunk() throws Exception {
public ByteBuf readChunk(ByteBufAllocator allocator) throws Exception {
synchronized (closeLock) {
if (fd.valid()) {
if (manageOsCache && readaheadPool != null) {
readaheadRequest = readaheadPool
.readaheadStream(
identifier, fd, getCurrentOffset(), readaheadLength,
getEndOffset(), readaheadRequest);
identifier, fd, currentOffset(), readaheadLength,
endOffset(), readaheadRequest);
}
return super.nextChunk();
return super.readChunk(allocator);
} else {
return null;
}
Expand All @@ -88,12 +89,12 @@ public void close() throws Exception {
readaheadRequest = null;
}
if (fd.valid() &&
manageOsCache && getEndOffset() - getStartOffset() > 0) {
manageOsCache && endOffset() - startOffset() > 0) {
try {
NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
identifier,
fd,
getStartOffset(), getEndOffset() - getStartOffset(),
startOffset(), endOffset() - startOffset(),
POSIX_FADV_DONTNEED);
} catch (Throwable t) {
LOG.warn("Failed to manage OS cache for " + identifier +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.nio.channels.FileChannel;
import java.nio.channels.WritableByteChannel;

import io.netty.channel.DefaultFileRegion;
import org.apache.hadoop.io.ReadaheadPool;
import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
import org.apache.hadoop.io.nativeio.NativeIO;
Expand All @@ -33,8 +34,6 @@

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

import org.jboss.netty.channel.DefaultFileRegion;

import org.apache.hadoop.classification.VisibleForTesting;

public class FadvisedFileRegion extends DefaultFileRegion {
Expand Down Expand Up @@ -77,8 +76,8 @@ public long transferTo(WritableByteChannel target, long position)
throws IOException {
if (readaheadPool != null && readaheadLength > 0) {
readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
getPosition() + position, readaheadLength,
getPosition() + getCount(), readaheadRequest);
position() + position, readaheadLength,
position() + count(), readaheadRequest);
}

if(this.shuffleTransferToAllowed) {
Expand Down Expand Up @@ -147,22 +146,22 @@ long customShuffleTransfer(WritableByteChannel target, long position)


@Override
public void releaseExternalResources() {
protected void deallocate() {
if (readaheadRequest != null) {
readaheadRequest.cancel();
}
super.releaseExternalResources();
super.deallocate();
}

/**
* Call when the transfer completes successfully so we can advise the OS that
* we don't need the region to be cached anymore.
*/
public void transferSuccessful() {
if (manageOsCache && getCount() > 0) {
if (manageOsCache && count() > 0) {
try {
NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
fd, getPosition(), getCount(), POSIX_FADV_DONTNEED);
fd, position(), count(), POSIX_FADV_DONTNEED);
} catch (Throwable t) {
LOG.warn("Failed to manage OS cache for " + identifier, t);
}
Expand Down
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";
}
}
}
Loading