|
@@ -23,6 +23,8 @@ import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
|
|
import static org.apache.hadoop.test.MockitoMaker.make;
|
|
|
import static org.apache.hadoop.test.MockitoMaker.stub;
|
|
|
import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
|
|
|
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
|
|
|
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assume.assumeTrue;
|
|
|
|
|
@@ -39,6 +41,7 @@ import java.nio.ByteBuffer;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.zip.CheckedOutputStream;
|
|
|
import java.util.zip.Checksum;
|
|
|
|
|
@@ -69,17 +72,24 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Cont
|
|
|
import org.jboss.netty.channel.Channel;
|
|
|
import org.jboss.netty.channel.ChannelFuture;
|
|
|
import org.jboss.netty.channel.ChannelHandlerContext;
|
|
|
+import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
|
|
|
import org.jboss.netty.handler.codec.http.HttpRequest;
|
|
|
import org.jboss.netty.handler.codec.http.HttpResponse;
|
|
|
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
-
|
|
|
+import org.mortbay.jetty.HttpHeaders;
|
|
|
|
|
|
public class TestShuffleHandler {
|
|
|
static final long MiB = 1024 * 1024;
|
|
|
private static final Log LOG = LogFactory.getLog(TestShuffleHandler.class);
|
|
|
|
|
|
+ /**
|
|
|
+ * Test the validation of ShuffleHandler's meta-data's serialization and
|
|
|
+ * de-serialization.
|
|
|
+ *
|
|
|
+ * @throws Exception exception
|
|
|
+ */
|
|
|
@Test (timeout = 10000)
|
|
|
public void testSerializeMeta() throws Exception {
|
|
|
assertEquals(1, ShuffleHandler.deserializeMetaData(
|
|
@@ -90,6 +100,11 @@ public class TestShuffleHandler {
|
|
|
ShuffleHandler.serializeMetaData(8080)));
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Validate shuffle connection and input/output metrics.
|
|
|
+ *
|
|
|
+ * @throws Exception exception
|
|
|
+ */
|
|
|
@Test (timeout = 10000)
|
|
|
public void testShuffleMetrics() throws Exception {
|
|
|
MetricsSystem ms = new MetricsSystemImpl();
|
|
@@ -120,6 +135,11 @@ public class TestShuffleHandler {
|
|
|
assertGauge("ShuffleConnections", connections, rb);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Verify client prematurely closing a connection.
|
|
|
+ *
|
|
|
+ * @throws Exception exception.
|
|
|
+ */
|
|
|
@Test (timeout = 10000)
|
|
|
public void testClientClosesConnection() throws Exception {
|
|
|
final ArrayList<Throwable> failures = new ArrayList<Throwable>(1);
|
|
@@ -130,6 +150,20 @@ public class TestShuffleHandler {
|
|
|
protected Shuffle getShuffle(Configuration conf) {
|
|
|
// replace the shuffle handler with one stubbed for testing
|
|
|
return new Shuffle(conf) {
|
|
|
+ @Override
|
|
|
+ protected MapOutputInfo getMapOutputInfo(String base, String mapId,
|
|
|
+ int reduce, String user) throws IOException {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ protected void populateHeaders(List<String> mapIds, String jobId,
|
|
|
+ String user, int reduce, HttpRequest request,
|
|
|
+ HttpResponse response, boolean keepAliveParam,
|
|
|
+ Map<String, MapOutputInfo> infoMap) throws IOException {
|
|
|
+ // Only set response headers and skip everything else
|
|
|
+ // send some dummy value for content-length
|
|
|
+ super.setResponseHeaders(response, keepAliveParam, 100);
|
|
|
+ }
|
|
|
@Override
|
|
|
protected void verifyRequest(String appid, ChannelHandlerContext ctx,
|
|
|
HttpRequest request, HttpResponse response, URL requestUri)
|
|
@@ -137,7 +171,8 @@ public class TestShuffleHandler {
|
|
|
}
|
|
|
@Override
|
|
|
protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
|
|
|
- Channel ch, String user, String jobId, String mapId, int reduce)
|
|
|
+ Channel ch, String user, String mapId, int reduce,
|
|
|
+ MapOutputInfo info)
|
|
|
throws IOException {
|
|
|
// send a shuffle header and a lot of data down the channel
|
|
|
// to trigger a broken pipe
|
|
@@ -147,7 +182,7 @@ public class TestShuffleHandler {
|
|
|
header.write(dob);
|
|
|
ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
|
|
|
dob = new DataOutputBuffer();
|
|
|
- for (int i=0; i<100000; ++i) {
|
|
|
+ for (int i = 0; i < 100000; ++i) {
|
|
|
header.write(dob);
|
|
|
}
|
|
|
return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
|
|
@@ -187,6 +222,7 @@ public class TestShuffleHandler {
|
|
|
conn.connect();
|
|
|
DataInputStream input = new DataInputStream(conn.getInputStream());
|
|
|
Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
|
|
|
+ Assert.assertEquals("close", conn.getHeaderField(HttpHeaders.CONNECTION));
|
|
|
ShuffleHeader header = new ShuffleHeader();
|
|
|
header.readFields(input);
|
|
|
input.close();
|
|
@@ -196,6 +232,147 @@ public class TestShuffleHandler {
|
|
|
failures.size() == 0);
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 10000)
|
|
|
+ public void testKeepAlive() throws Exception {
|
|
|
+ final ArrayList<Throwable> failures = new ArrayList<Throwable>(1);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
|
|
|
+ conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true);
|
|
|
+ // try setting to -ve keep alive timeout.
|
|
|
+ conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, -100);
|
|
|
+ ShuffleHandler shuffleHandler = new ShuffleHandler() {
|
|
|
+ @Override
|
|
|
+ protected Shuffle getShuffle(final Configuration conf) {
|
|
|
+ // replace the shuffle handler with one stubbed for testing
|
|
|
+ return new Shuffle(conf) {
|
|
|
+ @Override
|
|
|
+ protected MapOutputInfo getMapOutputInfo(String base, String mapId,
|
|
|
+ int reduce, String user) throws IOException {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
|
|
|
+ HttpRequest request, HttpResponse response, URL requestUri)
|
|
|
+ throws IOException {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void populateHeaders(List<String> mapIds, String jobId,
|
|
|
+ String user, int reduce, HttpRequest request,
|
|
|
+ HttpResponse response, boolean keepAliveParam,
|
|
|
+ Map<String, MapOutputInfo> infoMap) throws IOException {
|
|
|
+ // Send some dummy data (populate content length details)
|
|
|
+ ShuffleHeader header =
|
|
|
+ new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1);
|
|
|
+ DataOutputBuffer dob = new DataOutputBuffer();
|
|
|
+ header.write(dob);
|
|
|
+ dob = new DataOutputBuffer();
|
|
|
+ for (int i = 0; i < 100000; ++i) {
|
|
|
+ header.write(dob);
|
|
|
+ }
|
|
|
+
|
|
|
+ long contentLength = dob.getLength();
|
|
|
+ // for testing purpose;
|
|
|
+ // disable connectinKeepAliveEnabled if keepAliveParam is available
|
|
|
+ if (keepAliveParam) {
|
|
|
+ connectionKeepAliveEnabled = false;
|
|
|
+ }
|
|
|
+
|
|
|
+ super.setResponseHeaders(response, keepAliveParam, contentLength);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
|
|
|
+ Channel ch, String user, String mapId, int reduce,
|
|
|
+ MapOutputInfo info) throws IOException {
|
|
|
+ HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
|
|
|
+
|
|
|
+ // send a shuffle header and a lot of data down the channel
|
|
|
+ // to trigger a broken pipe
|
|
|
+ ShuffleHeader header =
|
|
|
+ new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1);
|
|
|
+ DataOutputBuffer dob = new DataOutputBuffer();
|
|
|
+ header.write(dob);
|
|
|
+ ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
|
|
|
+ dob = new DataOutputBuffer();
|
|
|
+ for (int i = 0; i < 100000; ++i) {
|
|
|
+ header.write(dob);
|
|
|
+ }
|
|
|
+ return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void sendError(ChannelHandlerContext ctx,
|
|
|
+ HttpResponseStatus status) {
|
|
|
+ if (failures.size() == 0) {
|
|
|
+ failures.add(new Error());
|
|
|
+ ctx.getChannel().close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void sendError(ChannelHandlerContext ctx, String message,
|
|
|
+ HttpResponseStatus status) {
|
|
|
+ if (failures.size() == 0) {
|
|
|
+ failures.add(new Error());
|
|
|
+ ctx.getChannel().close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ }
|
|
|
+ };
|
|
|
+ shuffleHandler.init(conf);
|
|
|
+ shuffleHandler.start();
|
|
|
+
|
|
|
+ String shuffleBaseURL = "http://127.0.0.1:"
|
|
|
+ + shuffleHandler.getConfig().get(
|
|
|
+ ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY);
|
|
|
+ URL url =
|
|
|
+ new URL(shuffleBaseURL + "/mapOutput?job=job_12345_1&reduce=1&"
|
|
|
+ + "map=attempt_12345_1_m_1_0");
|
|
|
+ HttpURLConnection conn = (HttpURLConnection) url.openConnection();
|
|
|
+ conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
|
|
|
+ ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
|
|
|
+ conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
|
|
|
+ ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
|
|
|
+ conn.connect();
|
|
|
+ DataInputStream input = new DataInputStream(conn.getInputStream());
|
|
|
+ Assert.assertEquals(HttpHeaders.KEEP_ALIVE,
|
|
|
+ conn.getHeaderField(HttpHeaders.CONNECTION));
|
|
|
+ Assert.assertEquals("timeout=1",
|
|
|
+ conn.getHeaderField(HttpHeaders.KEEP_ALIVE));
|
|
|
+ Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
|
|
|
+ ShuffleHeader header = new ShuffleHeader();
|
|
|
+ header.readFields(input);
|
|
|
+ input.close();
|
|
|
+
|
|
|
+ // For keepAlive via URL
|
|
|
+ url =
|
|
|
+ new URL(shuffleBaseURL + "/mapOutput?job=job_12345_1&reduce=1&"
|
|
|
+ + "map=attempt_12345_1_m_1_0&keepAlive=true");
|
|
|
+ conn = (HttpURLConnection) url.openConnection();
|
|
|
+ conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
|
|
|
+ ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
|
|
|
+ conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
|
|
|
+ ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
|
|
|
+ conn.connect();
|
|
|
+ input = new DataInputStream(conn.getInputStream());
|
|
|
+ Assert.assertEquals(HttpHeaders.KEEP_ALIVE,
|
|
|
+ conn.getHeaderField(HttpHeaders.CONNECTION));
|
|
|
+ Assert.assertEquals("timeout=1",
|
|
|
+ conn.getHeaderField(HttpHeaders.KEEP_ALIVE));
|
|
|
+ Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
|
|
|
+ header = new ShuffleHeader();
|
|
|
+ header.readFields(input);
|
|
|
+ input.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * simulate a reducer that sends an invalid shuffle-header - sometimes a wrong
|
|
|
+ * header_name and sometimes a wrong version
|
|
|
+ *
|
|
|
+ * @throws Exception exception
|
|
|
+ */
|
|
|
@Test (timeout = 10000)
|
|
|
public void testIncompatibleShuffleVersion() throws Exception {
|
|
|
final int failureNum = 3;
|
|
@@ -224,7 +401,12 @@ public class TestShuffleHandler {
|
|
|
shuffleHandler.stop();
|
|
|
shuffleHandler.close();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Validate the limit on number of shuffle connections.
|
|
|
+ *
|
|
|
+ * @throws Exception exception
|
|
|
+ */
|
|
|
@Test (timeout = 10000)
|
|
|
public void testMaxConnections() throws Exception {
|
|
|
|
|
@@ -236,14 +418,29 @@ public class TestShuffleHandler {
|
|
|
protected Shuffle getShuffle(Configuration conf) {
|
|
|
// replace the shuffle handler with one stubbed for testing
|
|
|
return new Shuffle(conf) {
|
|
|
+ @Override
|
|
|
+ protected MapOutputInfo getMapOutputInfo(String base, String mapId,
|
|
|
+ int reduce, String user) throws IOException {
|
|
|
+ // Do nothing.
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ protected void populateHeaders(List<String> mapIds, String jobId,
|
|
|
+ String user, int reduce, HttpRequest request,
|
|
|
+ HttpResponse response, boolean keepAliveParam,
|
|
|
+ Map<String, MapOutputInfo> infoMap) throws IOException {
|
|
|
+ // Do nothing.
|
|
|
+ }
|
|
|
@Override
|
|
|
protected void verifyRequest(String appid, ChannelHandlerContext ctx,
|
|
|
HttpRequest request, HttpResponse response, URL requestUri)
|
|
|
throws IOException {
|
|
|
+ // Do nothing.
|
|
|
}
|
|
|
@Override
|
|
|
protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
|
|
|
- Channel ch, String user, String jobId, String mapId, int reduce)
|
|
|
+ Channel ch, String user, String mapId, int reduce,
|
|
|
+ MapOutputInfo info)
|
|
|
throws IOException {
|
|
|
// send a shuffle header and a lot of data down the channel
|
|
|
// to trigger a broken pipe
|
|
@@ -308,7 +505,13 @@ public class TestShuffleHandler {
|
|
|
|
|
|
shuffleHandler.stop();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Validate the ownership of the map-output files being pulled in. The
|
|
|
+ * local-file-system owner of the file should match the user component in the
|
|
|
+ *
|
|
|
+ * @throws Exception exception
|
|
|
+ */
|
|
|
@Test(timeout = 100000)
|
|
|
public void testMapFileAccess() throws IOException {
|
|
|
// This will run only in NativeIO is enabled as SecureIOUtils need it
|
|
@@ -323,7 +526,7 @@ public class TestShuffleHandler {
|
|
|
TestShuffleHandler.class.getSimpleName() + "LocDir").getAbsoluteFile();
|
|
|
conf.set(YarnConfiguration.NM_LOCAL_DIRS, absLogDir.getAbsolutePath());
|
|
|
ApplicationId appId = ApplicationId.newInstance(12345, 1);
|
|
|
- System.out.println(appId.toString());
|
|
|
+ LOG.info(appId.toString());
|
|
|
String appAttemptId = "attempt_12345_1_m_1_0";
|
|
|
String user = "randomUser";
|
|
|
String reducerId = "0";
|
|
@@ -341,6 +544,7 @@ public class TestShuffleHandler {
|
|
|
protected void verifyRequest(String appid, ChannelHandlerContext ctx,
|
|
|
HttpRequest request, HttpResponse response, URL requestUri)
|
|
|
throws IOException {
|
|
|
+ // Do nothing.
|
|
|
}
|
|
|
|
|
|
};
|
|
@@ -393,7 +597,7 @@ public class TestShuffleHandler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public static void createShuffleHandlerFiles(File logDir, String user,
|
|
|
+ private static void createShuffleHandlerFiles(File logDir, String user,
|
|
|
String appId, String appAttemptId, Configuration conf,
|
|
|
List<File> fileMap) throws IOException {
|
|
|
String attemptDir =
|
|
@@ -412,8 +616,8 @@ public class TestShuffleHandler {
|
|
|
createMapOutputFile(mapOutputFile, conf);
|
|
|
}
|
|
|
|
|
|
- public static void
|
|
|
- createMapOutputFile(File mapOutputFile, Configuration conf)
|
|
|
+ private static void
|
|
|
+ createMapOutputFile(File mapOutputFile, Configuration conf)
|
|
|
throws IOException {
|
|
|
FileOutputStream out = new FileOutputStream(mapOutputFile);
|
|
|
out.write("Creating new dummy map output file. Used only for testing"
|
|
@@ -422,7 +626,7 @@ public class TestShuffleHandler {
|
|
|
out.close();
|
|
|
}
|
|
|
|
|
|
- public static void createIndexFile(File indexFile, Configuration conf)
|
|
|
+ private static void createIndexFile(File indexFile, Configuration conf)
|
|
|
throws IOException {
|
|
|
if (indexFile.exists()) {
|
|
|
System.out.println("Deleting existing file");
|