@@ -70,6 +73,7 @@
@InterfaceAudience.Private
class BPServiceActor implements Runnable {
+ static final XTrace.Logger xtrace = XTrace.getLogger(BPServiceActor.class);
static final Log LOG = DataNode.LOG;
final InetSocketAddress nnAddr;
@@ -313,6 +317,7 @@ void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) {
pendingIncrementalBR.put(
bInfo.getBlock().getBlockId(), bInfo);
pendingReceivedRequests++;
+ xtrace.log("Triggering block report to namenode");
pendingIncrementalBR.notifyAll();
}
}
@@ -522,7 +527,15 @@ private void offerService() throws Exception {
//
lastHeartbeat = startTime;
if (!dn.areHeartbeatsDisabledForTests()) {
- HeartbeatResponse resp = sendHeartBeat();
+ HDFSBackgroundTask.HEARTBEAT.start();
+ long begin = System.nanoTime();
+ HeartbeatResponse resp;
+ try {
+ resp = sendHeartBeat();
+ } finally {
+ HDFSBackgroundTask.HEARTBEAT.end(System.nanoTime() - begin);
+ }
+
assert resp != null;
dn.getMetrics().addHeartbeat(now() - startTime);
@@ -535,6 +548,7 @@ private void offerService() throws Exception {
bpos.updateActorStatesFromHeartbeat(
this, resp.getNameNodeHaState());
+ begin = System.nanoTime();
long startProcessCommands = now();
if (!processCommand(resp.getCommands()))
continue;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
index 8558e95d3f67..36d8a140eac3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
@@ -41,6 +41,9 @@
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.util.Daemon;
+import edu.brown.cs.systems.resourcetracing.backgroundtasks.HDFSBackgroundTask;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Manages storage for the set of BlockPoolSlices which share a particular
* block pool id, on this DataNode.
@@ -421,6 +424,9 @@ void doRollback(StorageDirectory bpSd, NamespaceInfo nsInfo)
* that holds the snapshot.
*/
void doFinalize(File dnCurDir) throws IOException {
+ HDFSBackgroundTask.FINALIZE.start();
+ final long begin = System.nanoTime();
+
File bpRoot = getBpRoot(blockpoolID, dnCurDir);
StorageDirectory bpSd = new StorageDirectory(bpRoot);
// block pool level previous directory
@@ -446,6 +452,8 @@ public void run() {
deleteDir(tmpDir);
} catch (IOException ex) {
LOG.error("Finalize upgrade for " + dataDirPath + " failed.", ex);
+ } finally {
+ HDFSBackgroundTask.FINALIZE.end(System.nanoTime() - begin);
}
LOG.info("Finalize upgrade for " + dataDirPath + " is complete.");
}
@@ -455,6 +463,8 @@ public String toString() {
return "Finalize " + dataDirPath;
}
}).start();
+
+ XTrace.stop();
}
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index d75d5a123b78..53922c2f63d3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -53,15 +53,23 @@
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.StringUtils;
+import edu.brown.cs.systems.resourcethrottling.LocalThrottlingPoints;
+import edu.brown.cs.systems.resourcethrottling.ThrottlingPoint;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/** A class that receives a block and writes to its own disk, meanwhile
* may copies it to another site. If a throttler is provided,
* streaming throttling is also supported.
**/
class BlockReceiver implements Closeable {
+ public static final XTrace.Logger xtrace = XTrace.getLogger(BlockReceiver.class);
+ public static final ThrottlingPoint throttlingpoint = LocalThrottlingPoints.getThrottlingPoint("BlockReceiver");
public static final Log LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog;
- private static final long CACHE_DROP_LAG_BYTES = 8 * 1024 * 1024;
+// private static final long CACHE_DROP_LAG_BYTES = 8 * 1024 * 1024;
+ private static final long CACHE_DROP_LAG_BYTES = 64 * 1024;
private DataInputStream in = null; // from where data are read
private DataChecksum clientChecksum; // checksum used by client
@@ -264,6 +272,7 @@ public void close() throws IOException {
// close checksum file
try {
if (checksumOut != null) {
+ xtrace.log("Closing checksum file");
long flushStartNanos = System.nanoTime();
checksumOut.flush();
long flushEndNanos = System.nanoTime();
@@ -276,6 +285,7 @@ public void close() throws IOException {
measuredFlushTime = true;
checksumOut.close();
checksumOut = null;
+ xtrace.log("BlockReceiver", "Checksum file closed");
}
} catch(IOException e) {
ioe = e;
@@ -286,6 +296,7 @@ public void close() throws IOException {
// close block file
try {
if (out != null) {
+ xtrace.log("Closing block file");
long flushStartNanos = System.nanoTime();
out.flush();
long flushEndNanos = System.nanoTime();
@@ -298,6 +309,7 @@ public void close() throws IOException {
measuredFlushTime = true;
out.close();
out = null;
+ xtrace.log("Block file closed");
}
} catch (IOException e) {
ioe = e;
@@ -336,6 +348,7 @@ void flushOrSync(boolean isSync) throws IOException {
long flushStartNanos = System.nanoTime();
out.flush();
long flushEndNanos = System.nanoTime();
+ xtrace.log("Flushed file output stream");
if (isSync && (out instanceof FileOutputStream)) {
long fsyncStartNanos = flushEndNanos;
((FileOutputStream)out).getChannel().force(true);
@@ -425,6 +438,7 @@ private boolean shouldVerifyChecksum() {
* returns the number of data bytes that the packet has.
*/
private int receivePacket() throws IOException {
+
// read the next packet
packetReceiver.receiveNextPacket(in);
@@ -558,6 +572,7 @@ private int receivePacket() throws IOException {
int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
// Write data to disk.
+ xtrace.log("Writing packet to file output stream", "start", startByteToDisk, "size", numBytesToDisk);
out.write(dataBuf.array(), startByteToDisk, numBytesToDisk);
// If this is a partial chunk, then verify that this is the only
@@ -617,6 +632,9 @@ private int receivePacket() throws IOException {
throttler.throttle(len);
}
+ // Retro throttle
+ throttlingpoint.throttle();
+
return lastPacketInBlock?-1:len;
}
@@ -648,6 +666,9 @@ void receiveBlock(
DataOutputStream replyOut, // output to previous datanode
String mirrAddr, DataTransferThrottler throttlerArg,
DatanodeInfo[] downstreams) throws IOException {
+
+ xtrace.log("Receiving Block");
+ try { // xtrace try
syncOnClose = datanode.getDnConf().syncOnClose;
boolean responderClosed = false;
@@ -721,6 +742,10 @@ void receiveBlock(
responder = null;
}
}
+ xtrace.log("Finished receiving block");
+ } catch (IOException e) { // xtrace catch
+ xtrace.log("IOException receiving block", "Message", e.getMessage());
+ }
}
/** Cleanup a partial block
@@ -834,8 +859,8 @@ class PacketResponder implements Runnable, Closeable {
/** The type of this responder */
private final PacketResponderType type;
/** for log and error messages */
- private final String myString;
-
+ private final String myString;
+
@Override
public String toString() {
return myString;
@@ -899,7 +924,7 @@ public synchronized void close() {
running = false;
notifyAll();
}
-
+
/**
* Thread to process incoming acks.
* @see java.lang.Runnable#run()
@@ -923,11 +948,14 @@ public void run() {
&& !mirrorError) {
// read an ack from downstream datanode
ack.readFields(downstreamIn);
+ ack.joinXtraceContext();
ackRecvNanoTime = System.nanoTime();
if (LOG.isDebugEnabled()) {
LOG.debug(myString + " got " + ack);
}
+
seqno = ack.getSeqno();
+ xtrace.log("Received ack", "seqno", seqno);
}
if (seqno != PipelineAck.UNKOWN_SEQNO
|| type == PacketResponderType.LAST_IN_PIPELINE) {
@@ -943,7 +971,9 @@ public void run() {
break;
}
pkt = ackQueue.getFirst();
+ pkt.joinXtraceContext();
expected = pkt.seqno;
+
if (type == PacketResponderType.HAS_DOWNSTREAM_IN_PIPELINE
&& seqno != expected) {
throw new IOException(myString + "seqno: expected="
@@ -979,6 +1009,7 @@ public void run() {
// and wait for the client to shut down the pipeline
mirrorError = true;
LOG.info(myString, ioe);
+ xtrace.log("IOException from mirror", "Message", ioe.getMessage());
}
}
@@ -991,6 +1022,7 @@ public void run() {
* will detect that this datanode is bad, and rightly so.
*/
LOG.info(myString + ": Thread is interrupted.");
+ xtrace.log("Thread is interrupted");
running = false;
continue;
}
@@ -1043,6 +1075,7 @@ public void run() {
continue;
}
}
+ xtrace.log("Acknowledging packet", "seqno", expected, "AckTimeNanos", totalAckTimeNanos);
PipelineAck replyAck = new PipelineAck(expected, replies, totalAckTimeNanos);
if (replyAck.isSuccess() &&
@@ -1060,6 +1093,7 @@ public void run() {
removeAckHead();
// update bytes acked
}
+ xtrace.log("Packet ack sent.");
// terminate after sending response if this node detected
// a checksum error
if (myStatus == Status.ERROR_CHECKSUM) {
@@ -1070,6 +1104,7 @@ public void run() {
}
} catch (IOException e) {
LOG.warn("IOException in BlockReceiver.run(): ", e);
+ xtrace.log("IOException in BlockReceiver.run", "Message", e.getMessage());
if (running) {
try {
datanode.checkDiskError(e); // may throw an exception here
@@ -1113,6 +1148,9 @@ private static class Packet {
final long offsetInBlock;
final long ackEnqueueNanoTime;
final Status ackStatus;
+
+ /** Remember the XTrace context when this packet was created */
+ private final Context xtrace_context = XTrace.get();
Packet(long seqno, boolean lastPacketInBlock, long offsetInBlock,
long ackEnqueueNanoTime, Status ackStatus) {
@@ -1122,6 +1160,10 @@ private static class Packet {
this.ackEnqueueNanoTime = ackEnqueueNanoTime;
this.ackStatus = ackStatus;
}
+
+ public void joinXtraceContext() {
+ XTrace.join(xtrace_context);
+ }
@Override
public String toString() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index 0e1e35c73362..92ce31aa5f82 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -47,6 +47,10 @@
import com.google.common.base.Preconditions;
+import edu.brown.cs.systems.resourcethrottling.LocalThrottlingPoints;
+import edu.brown.cs.systems.resourcethrottling.ThrottlingPoint;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Reads a block from the disk and sends it to a recipient.
*
@@ -88,6 +92,8 @@
* no checksum error, it replies to DataNode with OP_STATUS_CHECKSUM_OK.
*/
class BlockSender implements java.io.Closeable {
+ static final XTrace.Logger xtrace = XTrace.getLogger(BlockSender.class);
+ static final ThrottlingPoint throttlingpoint = LocalThrottlingPoints.getThrottlingPoint("BlockSender");
static final Log LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog;
private static final boolean is32Bit =
@@ -142,6 +148,7 @@ class BlockSender implements java.io.Closeable {
// Cache-management related fields
private final long readaheadLength;
private boolean shouldDropCacheBehindRead;
+ private boolean shouldFadviseSequential;
private ReadaheadRequest curReadahead;
private long lastCacheDropOffset;
private static final long CACHE_DROP_INTERVAL_BYTES = 1024 * 1024; // 1MB
@@ -177,6 +184,7 @@ class BlockSender implements java.io.Closeable {
this.clientTraceFmt = clientTraceFmt;
this.readaheadLength = datanode.getDnConf().readaheadLength;
this.shouldDropCacheBehindRead = datanode.getDnConf().dropCacheBehindReads;
+ this.shouldFadviseSequential = datanode.getDnConf().fadviseSequential;
this.datanode = datanode;
if (verifyChecksum) {
@@ -226,7 +234,9 @@ class BlockSender implements java.io.Closeable {
*/
DataChecksum csum = null;
if (verifyChecksum || sendChecksum) {
+ xtrace.log("Getting metadata input stream");
final InputStream metaIn = datanode.data.getMetaDataInputStream(block);
+ xtrace.log("Got metadata input stream");
if (!corruptChecksumOk || metaIn != null) {
if (metaIn == null) {
//need checksum but meta-data not found
@@ -454,6 +464,8 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out,
int packetLen = dataLen + checksumDataLen + 4;
boolean lastDataPacket = offset + dataLen == endOffset && dataLen > 0;
+ xtrace.log("Sending packet", "dataLen", dataLen, "numChunks", numChunks, "packetLen", packetLen);
+
// The packet buffer is organized as follows:
// _______HHHHCCCCD?D?D?D?
// ^ ^
@@ -515,6 +527,7 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out,
// normal transfer
out.write(buf, headerOff, dataOff + dataLen - headerOff);
}
+ xtrace.log("Packet send complete");
} catch (IOException e) {
if (e instanceof SocketTimeoutException) {
/*
@@ -523,6 +536,7 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out,
* the socket open).
*/
LOG.info("exception: ", e);
+ xtrace.log("SocketTimeoutException");
} else {
/* Exception while writing to the client. Connection closure from
* the other end is mostly the case and we do not care much about
@@ -537,6 +551,7 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out,
if (!ioem.startsWith("Broken pipe") && !ioem.startsWith("Connection reset")) {
LOG.error("BlockSender.sendChunks() exception: ", e);
}
+ xtrace.log("Exception", "Message", ioem);
}
throw ioeToSocketException(e);
}
@@ -544,6 +559,9 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out,
if (throttler != null) { // rebalancing so throttle
throttler.throttle(packetLen);
}
+
+ // Retro throttle
+ throttlingpoint.throttle();
return dataLen;
}
@@ -625,17 +643,21 @@ public void verifyChecksum(final byte[] buf, final int dataOffset,
* @return total bytes read, including checksum data.
*/
long sendBlock(DataOutputStream out, OutputStream baseStream,
- DataTransferThrottler throttler) throws IOException {
+ DataTransferThrottler throttler) throws IOException {
if (out == null) {
throw new IOException( "out stream is null" );
}
+
+ xtrace.log("Sending Block", "BlockName", block.getBlockName());
+ try { // xtrace try
+
initialOffset = offset;
long totalRead = 0;
OutputStream streamForSendChunks = out;
lastCacheDropOffset = initialOffset;
- if (isLongRead() && blockInFd != null) {
+ if (isLongRead() && blockInFd != null && shouldFadviseSequential) {
// Advise that this file descriptor will be accessed sequentially.
NativeIO.POSIX.posixFadviseIfPossible(
blockInFd, 0, 0, NativeIO.POSIX.POSIX_FADV_SEQUENTIAL);
@@ -688,6 +710,9 @@ long sendBlock(DataOutputStream out, OutputStream baseStream,
}
sentEntireByteRange = true;
+ xtrace.log("Entire block sent", "totalRead", totalRead, "initialOffset", initialOffset);
+ } else {
+ xtrace.log("Block send interrupted", "totalRead", totalRead, "initialOffset", initialOffset);
}
} finally {
if (clientTraceFmt != null) {
@@ -698,6 +723,11 @@ long sendBlock(DataOutputStream out, OutputStream baseStream,
close();
}
return totalRead;
+
+ } catch (IOException e) { // xtrace catch
+ xtrace.log("IOException sending block", "Message", e.getMessage());
+ throw e;
+ }
}
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
index 1577d78eddc3..09c5061179f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
@@ -57,6 +57,7 @@ public class DNConf {
final boolean dropCacheBehindWrites;
final boolean syncBehindWrites;
final boolean dropCacheBehindReads;
+ final boolean fadviseSequential;
final boolean syncOnClose;
final boolean encryptDataTransfer;
final boolean connectToDnViaHostname;
@@ -103,6 +104,9 @@ public DNConf(Configuration conf) {
dropCacheBehindReads = conf.getBoolean(
DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY,
DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT);
+ fadviseSequential = conf.getBoolean(
+ DFSConfigKeys.DFS_DATANODE_FADVISE_SEQUENTIAL,
+ DFSConfigKeys.DFS_DATANODE_FADVISE_SEQUENTIAL_DEFAULT);
connectToDnViaHostname = conf.getBoolean(
DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME,
DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index ea016413df0c..26f5242b15d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -184,6 +184,10 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingService;
+import edu.brown.cs.systems.pivottracing.PivotTracingPubSub;
+import edu.brown.cs.systems.resourcetracing.backgroundtasks.HDFSBackgroundTask;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**********************************************************
* DataNode is a class (and program) that stores a set of
* blocks for a DFS deployment. A single deployment can
@@ -219,6 +223,7 @@
public class DataNode extends Configured
implements InterDatanodeProtocol, ClientDatanodeProtocol,
DataNodeMXBean {
+ public static final XTrace.Logger xtrace = XTrace.getLogger(DataNode.class);
public static final Log LOG = LogFactory.getLog(DataNode.class);
static{
@@ -1544,6 +1549,9 @@ private class DataTransfer implements Runnable {
*/
@Override
public void run() {
+ HDFSBackgroundTask.REPLICATION.start();
+ long begin = System.nanoTime();
+
xmitsInProgress.getAndIncrement();
Socket sock = null;
DataOutputStream out = null;
@@ -1635,6 +1643,8 @@ public void run() {
IOUtils.closeStream(out);
IOUtils.closeStream(in);
IOUtils.closeSocket(sock);
+
+ HDFSBackgroundTask.REPLICATION.end(System.nanoTime() - begin);
}
}
}
@@ -1929,6 +1939,7 @@ public static void secureMain(String args[], SecureResources resources) {
}
public static void main(String args[]) {
+ PivotTracingPubSub.initialize();
if (DFSUtil.parseHelpArgument(args, DataNode.USAGE, System.out, true)) {
System.exit(0);
}
@@ -1945,11 +1956,15 @@ public Daemon recoverBlocks(
@Override
public void run() {
for(RecoveringBlock b : blocks) {
+ HDFSBackgroundTask.RECOVER.start();
+ long begin = System.nanoTime();
try {
logRecoverBlock(who, b);
recoverBlock(b);
} catch (IOException e) {
LOG.warn("recoverBlocks FAILED: " + b, e);
+ } finally {
+ HDFSBackgroundTask.RECOVER.end(System.nanoTime() - begin);
}
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 9d31ffa673e4..2c11998e2792 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -55,6 +55,9 @@
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DiskChecker;
+import edu.brown.cs.systems.resourcetracing.backgroundtasks.HDFSBackgroundTask;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Data storage information file.
*
@@ -570,6 +573,9 @@ void doRollback( StorageDirectory sd,
* Do nothing, if previous directory does not exist
*/
void doFinalize(StorageDirectory sd) throws IOException {
+ HDFSBackgroundTask.FINALIZE.start();
+ final long begin = System.nanoTime();
+
File prevDir = sd.getPreviousDir();
if (!prevDir.exists())
return; // already discarded
@@ -598,12 +604,15 @@ public void run() {
}
} catch(IOException ex) {
LOG.error("Finalize upgrade for " + dataDirPath + " failed", ex);
+ } finally {
+ HDFSBackgroundTask.FINALIZE.end(System.nanoTime() - begin);
}
LOG.info("Finalize upgrade for " + dataDirPath + " is complete");
}
@Override
public String toString() { return "Finalize " + dataDirPath; }
}).start();
+ XTrace.stop();
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index b08959dbe2d1..df9c3cb1d142 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -18,11 +18,11 @@
package org.apache.hadoop.hdfs.server.datanode;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR;
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
-import static org.apache.hadoop.util.Time.now;
import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
+import static org.apache.hadoop.util.Time.now;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
@@ -46,10 +46,11 @@
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor.InvalidMagicNumberException;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
import org.apache.hadoop.hdfs.protocol.datatransfer.Receiver;
@@ -76,11 +77,14 @@
import com.google.protobuf.ByteString;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Thread for processing incoming/outgoing data stream.
*/
class DataXceiver extends Receiver implements Runnable {
+ public static final XTrace.Logger xtrace = XTrace.getLogger(DataXceiver.class);
public static final Log LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog;
@@ -220,12 +224,15 @@ public void run() {
opStartTime = now();
processOp(op);
++opsProcessed;
+ XTrace.stop();
} while (!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0);
} catch (Throwable t) {
LOG.error(datanode.getDisplayName() + ":DataXceiver error processing " +
((op == null) ? "unknown" : op.name()) + " operation " +
" src: " + remoteAddress +
" dest: " + localAddress, t);
+
+ xtrace.log("Error processing", "Message", t.getMessage());
} finally {
if (LOG.isDebugEnabled()) {
LOG.debug(datanode.getDisplayName() + ":Number of active connections is: "
@@ -241,6 +248,8 @@ public void run() {
public void requestShortCircuitFds(final ExtendedBlock blk,
final Token token,
int maxVersion) throws IOException {
+ xtrace.log("DataNode RequestShortCircuitFds");
+
updateCurrentThreadName("Passing file descriptors for block " + blk);
BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
FileInputStream fis[] = null;
@@ -267,6 +276,7 @@ public void requestShortCircuitFds(final ExtendedBlock blk,
bld.setMessage(e.getMessage());
}
try {
+ XTraceProtoUtils.setXtrace(bld);
bld.build().writeDelimitedTo(socketOut);
if (fis != null) {
FileDescriptor fds[] = new FileDescriptor[fis.length];
@@ -300,6 +310,7 @@ public void readBlock(final ExtendedBlock block,
final long blockOffset,
final long length,
final boolean sendChecksum) throws IOException {
+ xtrace.log("DataNode ReadBlock");
previousOpClientName = clientName;
OutputStream baseStream = getOutputStream();
@@ -321,6 +332,7 @@ public void readBlock(final ExtendedBlock block,
remoteAddress;
updateCurrentThreadName("Sending block " + block);
+ xtrace.log("Creating BlockSender");
try {
try {
blockSender = new BlockSender(block, blockOffset, length,
@@ -332,17 +344,22 @@ public void readBlock(final ExtendedBlock block,
throw e;
}
+ xtrace.log("Writing RPC result");
+
// send op status
writeSuccessWithChecksumInfo(blockSender, new DataOutputStream(getOutputStream()));
+ xtrace.log("Sending Block");
long read = blockSender.sendBlock(out, baseStream, null); // send data
-
+ xtrace.log("Sent Block");
+
if (blockSender.didSendEntireByteRange()) {
// If we sent the entire range, then we should expect the client
// to respond with a Status enum.
try {
ClientReadStatusProto stat = ClientReadStatusProto.parseFrom(
PBHelper.vintPrefixed(in));
+ XTraceProtoUtils.join(stat);
if (!stat.hasStatus()) {
LOG.warn("Client " + peer.getRemoteAddressString() +
" did not send a valid status code after reading. " +
@@ -353,8 +370,10 @@ public void readBlock(final ExtendedBlock block,
LOG.debug("Error reading client status response. Will close connection.", ioe);
IOUtils.closeStream(out);
}
+ xtrace.log("Received RPC status");
} else {
IOUtils.closeStream(out);
+ xtrace.log("Closed Stream");
}
datanode.metrics.incrBytesRead((int) read);
datanode.metrics.incrBlocksRead();
@@ -375,6 +394,8 @@ public void readBlock(final ExtendedBlock block,
throw ioe;
} finally {
IOUtils.closeStream(blockSender);
+
+ xtrace.log("ReadBlock Complete");
}
//update metrics
@@ -394,6 +415,7 @@ public void writeBlock(final ExtendedBlock block,
final long maxBytesRcvd,
final long latestGenerationStamp,
DataChecksum requestedChecksum) throws IOException {
+ xtrace.log("DataNode WriteBlock");
previousOpClientName = clientname;
updateCurrentThreadName("Receiving block " + block);
final boolean isDatanode = clientname.length() == 0;
@@ -467,6 +489,7 @@ public void writeBlock(final ExtendedBlock block,
if (LOG.isDebugEnabled()) {
LOG.debug("Connecting to datanode " + mirrorNode);
}
+ xtrace.log("Connecting to mirror node", "mirrorNode", mirrorNode);
mirrorTarget = NetUtils.createSocketAddr(mirrorNode);
mirrorSock = datanode.newSocket();
try {
@@ -507,17 +530,20 @@ public void writeBlock(final ExtendedBlock block,
BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(mirrorIn));
mirrorInStatus = connectAck.getStatus();
firstBadLink = connectAck.getFirstBadLink();
+ XTraceProtoUtils.join(connectAck);
if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) {
LOG.info("Datanode " + targets.length +
" got response for connect ack " +
" from downstream datanode with firstbadlink as " +
firstBadLink);
}
+ xtrace.log("Got response for connect ack from downstream datanode");
}
} catch (IOException e) {
+ xtrace.log("Exception transferring block");
if (isClient) {
- BlockOpResponseProto.newBuilder()
+ XTraceProtoUtils.newBlockOpResponseProtoBuilder()
.setStatus(ERROR)
// NB: Unconditionally using the xfer addr w/o hostname
.setFirstBadLink(targets[0].getXferAddr())
@@ -545,12 +571,13 @@ public void writeBlock(final ExtendedBlock block,
// send connect-ack to source for clients and not transfer-RBW/Finalized
if (isClient && !isTransfer) {
+ xtrace.log("Forwarding connect ack to upstream");
if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) {
LOG.info("Datanode " + targets.length +
" forwarding connect ack to upstream firstbadlink is " +
firstBadLink);
}
- BlockOpResponseProto.newBuilder()
+ XTraceProtoUtils.newBlockOpResponseProtoBuilder()
.setStatus(mirrorInStatus)
.setFirstBadLink(firstBadLink)
.build()
@@ -569,6 +596,7 @@ public void writeBlock(final ExtendedBlock block,
if (LOG.isTraceEnabled()) {
LOG.trace("TRANSFER: send close-ack");
}
+ xtrace.log("Sending close ack");
writeResponse(SUCCESS, null, replyOut);
}
}
@@ -601,6 +629,8 @@ public void writeBlock(final ExtendedBlock block,
IOUtils.closeStream(replyOut);
IOUtils.closeSocket(mirrorSock);
IOUtils.closeStream(blockReceiver);
+
+ xtrace.log("WriteBlock Complete");
}
//update metrics
@@ -613,6 +643,7 @@ public void transferBlock(final ExtendedBlock blk,
final Token blockToken,
final String clientName,
final DatanodeInfo[] targets) throws IOException {
+ xtrace.log("DataNode TransferBlock");
checkAccess(socketOut, true, blk, blockToken,
Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
previousOpClientName = clientName;
@@ -631,6 +662,7 @@ public void transferBlock(final ExtendedBlock blk,
@Override
public void blockChecksum(final ExtendedBlock block,
final Token blockToken) throws IOException {
+ xtrace.log("DataNode BlockChecksum");
final DataOutputStream out = new DataOutputStream(
getOutputStream());
checkAccess(out, true, block, blockToken,
@@ -659,7 +691,7 @@ public void blockChecksum(final ExtendedBlock block,
}
//write reply
- BlockOpResponseProto.newBuilder()
+ XTraceProtoUtils.newBlockOpResponseProtoBuilder()
.setStatus(SUCCESS)
.setChecksumResponse(OpBlockChecksumResponseProto.newBuilder()
.setBytesPerCrc(bytesPerCRC)
@@ -683,6 +715,7 @@ public void blockChecksum(final ExtendedBlock block,
@Override
public void copyBlock(final ExtendedBlock block,
final Token blockToken) throws IOException {
+ xtrace.log("DataNode CopyBlock");
updateCurrentThreadName("Copying block " + block);
// Read in the header
if (datanode.isBlockTokenEnabled) {
@@ -758,6 +791,7 @@ public void replaceBlock(final ExtendedBlock block,
final Token blockToken,
final String delHint,
final DatanodeInfo proxySource) throws IOException {
+ xtrace.log("DataNode ReplaceBlock");
updateCurrentThreadName("Replacing block " + block + " from " + delHint);
/* read header */
@@ -827,6 +861,7 @@ public void replaceBlock(final ExtendedBlock block,
BlockOpResponseProto copyResponse = BlockOpResponseProto.parseFrom(
PBHelper.vintPrefixed(proxyReply));
+ XTraceProtoUtils.join(copyResponse);
if (copyResponse.getStatus() != SUCCESS) {
if (copyResponse.getStatus() == ERROR_ACCESS_TOKEN) {
@@ -911,6 +946,7 @@ private static void writeResponse(Status status, String message, OutputStream ou
if (message != null) {
response.setMessage(message);
}
+ XTraceProtoUtils.setXtrace(response);
response.build().writeDelimitedTo(out);
out.flush();
}
@@ -923,7 +959,7 @@ private void writeSuccessWithChecksumInfo(BlockSender blockSender,
.setChunkOffset(blockSender.getOffset())
.build();
- BlockOpResponseProto response = BlockOpResponseProto.newBuilder()
+ BlockOpResponseProto response = XTraceProtoUtils.newBlockOpResponseProtoBuilder()
.setStatus(SUCCESS)
.setReadOpChecksumInfo(ckInfo)
.build();
@@ -955,6 +991,7 @@ private void checkAccess(OutputStream out, final boolean reply,
// NB: Unconditionally using the xfer addr w/o hostname
resp.setFirstBadLink(dnR.getXferAddr());
}
+ XTraceProtoUtils.setXtrace(resp);
resp.build().writeDelimitedTo(out);
out.flush();
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
index 806921d2263f..e14f8bd23899 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
@@ -32,6 +32,8 @@
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import edu.brown.cs.systems.resourcetracing.backgroundtasks.HDFSBackgroundTask;
+
/**
* This class is a container of multiple thread pools, each for a volume,
* so that we can schedule async disk operations easily.
@@ -153,11 +155,11 @@ synchronized void shutdown() {
* dfsUsed statistics accordingly.
*/
void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile,
- ExtendedBlock block) {
+ ExtendedBlock block, long begin) {
LOG.info("Scheduling " + block.getLocalBlock()
+ " file " + blockFile + " for deletion");
ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
- volume, blockFile, metaFile, block);
+ volume, blockFile, metaFile, block, begin);
execute(volume.getCurrentDir(), deletionTask);
}
@@ -169,13 +171,15 @@ class ReplicaFileDeleteTask implements Runnable {
final File blockFile;
final File metaFile;
final ExtendedBlock block;
+ final long begin;
ReplicaFileDeleteTask(FsVolumeImpl volume, File blockFile,
- File metaFile, ExtendedBlock block) {
+ File metaFile, ExtendedBlock block, long deleteBegin) {
this.volume = volume;
this.blockFile = blockFile;
this.metaFile = metaFile;
this.block = block;
+ this.begin = deleteBegin;
}
@Override
@@ -201,6 +205,7 @@ public void run() {
LOG.info("Deleted " + block.getBlockPoolId() + " "
+ block.getLocalBlock() + " file " + blockFile);
}
+ HDFSBackgroundTask.INVALIDATE.end(System.nanoTime() - begin);
}
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index ba0d2a873cc1..695fd026ea00 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -84,6 +84,9 @@
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Time;
+import edu.brown.cs.systems.resourcetracing.backgroundtasks.HDFSBackgroundTask;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**************************************************
* FSDataset manages a set of data blocks. Each block
* has a unique name and an extent on disk.
@@ -1096,6 +1099,8 @@ static void checkReplicaFiles(final ReplicaInfo r) throws IOException {
public void invalidate(String bpid, Block invalidBlks[]) throws IOException {
boolean error = false;
for (int i = 0; i < invalidBlks.length; i++) {
+ HDFSBackgroundTask.INVALIDATE.start();
+ long start = System.nanoTime();
final File f;
final FsVolumeImpl v;
synchronized (this) {
@@ -1146,7 +1151,8 @@ public void invalidate(String bpid, Block invalidBlks[]) throws IOException {
// Delete the block asynchronously to make sure we can do it fast enough
asyncDiskService.deleteAsync(v, f,
FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()),
- new ExtendedBlock(bpid, invalidBlks[i]));
+ new ExtendedBlock(bpid, invalidBlks[i]), start);
+ XTrace.stop();
}
if (error) {
throw new IOException("Error in deleting blocks.");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index e28c568ba947..db6d2c0809cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -162,7 +162,13 @@
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.*;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
+import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
@@ -173,14 +179,7 @@
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.Status;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
@@ -192,6 +191,12 @@
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.Status;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -231,6 +236,12 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
+import edu.brown.cs.systems.pubsub.PubSub;
+import edu.brown.cs.systems.pubsub.PubSubProtos.StringMessage;
+import edu.brown.cs.systems.pubsub.Subscriber;
+
+
+
/***************************************************
* FSNamesystem does the actual bookkeeping work for the
* DataNode.
@@ -3914,6 +3925,61 @@ void registerDatanode(DatanodeRegistration nodeReg) throws IOException {
String getRegistrationID() {
return Storage.getRegistrationID(dir.fsImage.getStorage());
}
+
+
+ private static final String replication_topic = "replication";
+
+ /**
+ * X-Trace: sends a pubsub message specifying the bytes per second allowed by replication
+ * @param bytes
+ */
+ public static void setReplicationTotalBytesPerSecond(long bytes) {
+ String command = "set:"+bytes;
+ PubSub.publish(replication_topic, StringMessage.newBuilder().setMessage(command).build());
+ }
+
+ public static void clearReplication() {
+ String command = "clear";
+ PubSub.publish(replication_topic, StringMessage.newBuilder().setMessage(command).build());
+ }
+
+ private static final int replication_command_timeout = 30000; // 30s timeout for replication commands
+ private ReplicationSubscriber replication_subscriber = new ReplicationSubscriber();
+ private volatile long last_replication_command = 0;
+ private volatile long replication_bps = 0;
+
+ private class ReplicationSubscriber extends Subscriber.Callback {
+
+ public ReplicationSubscriber() {
+ PubSub.subscribe(replication_topic, this);
+ }
+
+ @Override
+ protected void OnMessage(StringMessage msg) {
+ if (msg.hasMessage()) {
+ String command = msg.getMessage();
+ System.out.println("Replication command received: " + command);
+ try {
+ parseCommand(command);
+ } catch (Exception e) {
+ System.out.println(e.getClass().getName() + " parsing replication command " + msg.getMessage());
+ }
+ }
+ }
+
+ private void parseCommand(String command) {
+ String[] splits = command.split(":");
+ String cmd = splits[0];
+ if (cmd.equals("set")) {
+ replication_bps = Long.parseLong(splits[1]);
+ last_replication_command = System.currentTimeMillis();
+ } else if (cmd.equals("clear")) {
+ last_replication_command = 0;
+ replication_bps = 0;
+ }
+ }
+
+ }
/**
* The given node has reported in. This method should:
@@ -3934,9 +4000,14 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
try {
final int maxTransfer = blockManager.getMaxReplicationStreams()
- xmitsInProgress;
+ long maxBytes = Long.MAX_VALUE;
+ long numLiveNodes = blockManager.getDatanodeManager().getNumLiveDataNodes();
+ if ((System.currentTimeMillis()-last_replication_command) 0) {
+ maxBytes = replication_bps / numLiveNodes;
+ }
DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed,
- xceiverCount, maxTransfer, failedVolumes);
+ xceiverCount, maxTransfer, maxBytes, failedVolumes);
return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
} finally {
readUnlock();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 6b5e5dce5b4a..ed20a3c190c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -28,6 +28,7 @@
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -41,6 +42,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Trash;
+
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.util.ExitUtil.terminate;
import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
@@ -86,6 +88,8 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
+import edu.brown.cs.systems.pivottracing.PivotTracingPubSub;
+
/**********************************************************
* NameNode serves as both directory namespace manager and
* "inode table" for the Hadoop DFS. There is a single NameNode
@@ -1278,6 +1282,7 @@ protected String getNameServiceId(Configuration conf) {
/**
*/
public static void main(String argv[]) throws Exception {
+ PivotTracingPubSub.initialize();
if (DFSUtil.parseHelpArgument(argv, NameNode.USAGE, System.out, true)) {
System.exit(0);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
index 2049592d99f9..0b91a07ec6c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
@@ -47,6 +47,7 @@ message DataTransferEncryptorMessageProto {
message BaseHeaderProto {
required ExtendedBlockProto block = 1;
optional hadoop.common.TokenProto token = 2;
+ optional bytes xtrace = 3; // X-Trace metadata
}
message ClientOperationHeaderProto {
@@ -139,6 +140,7 @@ message PacketHeaderProto {
required bool lastPacketInBlock = 3;
required sfixed32 dataLen = 4;
optional bool syncBlock = 5 [default = false];
+ optional bytes xtrace = 6; // X-Trace metadata
}
enum Status {
@@ -156,6 +158,7 @@ message PipelineAckProto {
required sint64 seqno = 1;
repeated Status status = 2;
optional uint64 downstreamAckTimeNanos = 3 [default = 0];
+ optional bytes xtrace = 4; // X-Trace metadata
}
/**
@@ -192,6 +195,7 @@ message BlockOpResponseProto {
* read.
*/
optional uint32 shortCircuitAccessVersion = 6;
+ optional bytes xtrace = 7; // X-Trace metadata
}
/**
@@ -200,10 +204,12 @@ message BlockOpResponseProto {
*/
message ClientReadStatusProto {
required Status status = 1;
+ optional bytes xtrace = 2; // X-Trace metadata
}
message DNTransferAckProto {
required Status status = 1;
+ optional bytes xtrace = 2; // X-Trace metadata
}
message OpBlockChecksumResponseProto {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
index dd157282ae5f..278e9920fd41 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
@@ -49,10 +49,11 @@
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Runs the container task locally in a thread.
* Since all (sub)tasks share the same local directory, they must be executed
@@ -172,12 +173,14 @@ public void run() {
// (i.e., fork()), else will get weird failures when maps try to create/
// write same dirname or filename: no chdir() in Java
while (!Thread.currentThread().isInterrupted()) {
+ XTrace.stop();
try {
event = eventQueue.take();
} catch (InterruptedException e) { // mostly via T_KILL? JOB_KILL?
LOG.error("Returning, interrupted : " + e);
return;
}
+ event.joinContext();
LOG.info("Processing the event " + event.toString());
@@ -241,6 +244,9 @@ public void run() {
// (i.e., exit clumsily--but can never happen, so no worries!)
LOG.fatal("oopsie... this can never happen: "
+ StringUtils.stringifyException(ioe));
+ // TODO: X-Trace todo: join parent
+// XTraceContext.logEvent(LocalContainerLauncher.class, "LocalContainerLauncher", "Whoops. Fatal error.");
+// XTraceContext.joinParentProcess();
System.exit(-1);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
index aae95d47fe17..5ebdb20ccacb 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
@@ -55,6 +55,8 @@
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* This class is responsible for talking to the task umblical.
* It also converts all the old data structures
@@ -69,6 +71,7 @@ public class TaskAttemptListenerImpl extends CompositeService
private static final JvmTask TASK_FOR_INVALID_JVM = new JvmTask(null, true);
+ private static final XTrace.Logger xtrace = XTrace.getLogger(TaskAttemptListenerImpl.class);
private static final Log LOG = LogFactory.getLog(TaskAttemptListenerImpl.class);
private AppContext context;
@@ -434,8 +437,10 @@ public JvmTask getTask(JvmContext context) throws IOException {
// longer pending, and further request should ask it to exit.
org.apache.hadoop.mapred.Task task =
jvmIDToActiveAttemptMap.remove(wJvmID);
+ task.joinContext();
launchedJVMs.remove(wJvmID);
LOG.info("JVM with ID: " + jvmId + " given task: " + task.getTaskID());
+ xtrace.log("Sending task to JVM", "Task ID", task.getTaskID(), "JVM ID", jvmId);
jvmTask = new JvmTask(task, false);
}
}
@@ -449,7 +454,10 @@ public void registerPendingTask(
// when the jvm comes back to ask for Task.
// A JVM not present in this map is an illegal task/JVM.
- jvmIDToActiveAttemptMap.put(jvmID, task);
+ jvmIDToActiveAttemptMap.put(jvmID, task);
+
+ task.rememberContext();
+ xtrace.log("Task registered for JVM", "Task ID", task.getTaskID(), "JVM ID", jvmID);
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
index 6c1382d9e97e..5f7cddd6cd86 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
@@ -62,16 +62,26 @@
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.log4j.LogManager;
+import edu.brown.cs.systems.resourcethrottling.LocalThrottlingPoints;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* The main() for MapReduce task processes.
*/
class YarnChild {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(YarnChild.class);
private static final Log LOG = LogFactory.getLog(YarnChild.class);
static volatile TaskAttemptID taskid = null;
public static void main(String[] args) throws Throwable {
+ // Initialize throttling points
+ LocalThrottlingPoints.init();
+
+ // Load the XTrace context from the parent process
+ XTrace.set(System.getenv());
+ xtrace.log("YarnChild starting");
Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
LOG.debug("Child starting");
@@ -133,6 +143,7 @@ public TaskUmbilicalProtocol run() throws Exception {
myTask = umbilical.getTask(context);
}
if (myTask.shouldDie()) {
+ xtrace.log("Task has been instructed to die.");
return;
}
@@ -159,14 +170,17 @@ public TaskUmbilicalProtocol run() throws Exception {
public Object run() throws Exception {
// use job-specified working directory
FileSystem.get(job).setWorkingDirectory(job.getWorkingDirectory());
+ xtrace.log("Running task");
taskFinal.run(job, umbilical); // run the task
return null;
}
});
} catch (FSError e) {
+ xtrace.log("FSError from child: "+e.getClass().getName(), "Message", e.getMessage());
LOG.fatal("FSError from child", e);
umbilical.fsError(taskid, e.getMessage());
} catch (Exception exception) {
+ xtrace.log("Exception running child", "Exception", exception.getClass().getName(), "Message", exception.getMessage());
LOG.warn("Exception running child : "
+ StringUtils.stringifyException(exception));
try {
@@ -193,6 +207,7 @@ public Object run() throws Exception {
umbilical.fatalError(taskid, StringUtils.stringifyException(exception));
}
} catch (Throwable throwable) {
+ xtrace.log("Error running child", "Throwable", throwable.getClass().getName(), "Message", throwable.getMessage());
LOG.fatal("Error running child : "
+ StringUtils.stringifyException(throwable));
if (taskid != null) {
@@ -203,6 +218,10 @@ public Object run() throws Exception {
umbilical.fatalError(taskid, cause);
}
} finally {
+ xtrace.log("YarnChild exiting");
+ Thread.sleep(1000);
+ // TODO: xtrace join parent process
+// XTraceContext.joinParentProcess();
RPC.stopProxy(umbilical);
DefaultMetricsSystem.shutdown();
// Shutting down log4j of the child-vm...
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
index d3d3d0fa0158..82a62175ec8b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
@@ -56,6 +56,8 @@
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* The job history events get routed to this class. This class writes the Job
* history events to the DFS directly into a staging dir and then moved to a
@@ -259,12 +261,15 @@ public void run() {
eventCounter++;
}
+ XTrace.stop();
try {
event = eventQueue.take();
} catch (InterruptedException e) {
LOG.info("EventQueue take interrupted. Returning");
return;
}
+ event.joinContext();
+
// If an event has been removed from the queue. Handle it.
// The rest of the queue is handled via stop()
// Clear the interrupt status if it's set before calling handleEvent
@@ -480,6 +485,7 @@ private boolean isJobCompletionEvent(HistoryEvent historyEvent) {
}
protected void handleEvent(JobHistoryEvent event) {
+ event.joinContext();
synchronized (lock) {
// If this is JobSubmitted Event, setup the writer
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index 6ff498091727..1db554a2b365 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -135,6 +135,8 @@
import com.google.common.annotations.VisibleForTesting;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* The Map-Reduce Application Master.
* The state machine is encapsulated in the implementation of Job interface.
@@ -156,6 +158,7 @@
@SuppressWarnings("rawtypes")
public class MRAppMaster extends CompositeService {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(MRAppMaster.class);
private static final Log LOG = LogFactory.getLog(MRAppMaster.class);
/**
@@ -252,6 +255,11 @@ protected void serviceInit(final Configuration conf) throws Exception {
newApiCommitter = false;
jobId = MRBuilderUtils.newJobId(appAttemptID.getApplicationId(),
appAttemptID.getApplicationId().getId());
+
+ /* This will either generate a new task id, or pick up from an existing one
+ * if we had one passed to us or the xtrace environment variable was set */
+ xtrace.log("Starting job", "Job ID", jobId);
+
int numReduceTasks = conf.getInt(MRJobConfig.NUM_REDUCES, 0);
if ((numReduceTasks > 0 &&
conf.getBoolean("mapred.reducer.new-api", false)) ||
@@ -504,6 +512,8 @@ public void cleanupStagingDir() throws IOException {
* Exit call. Just in a function call to enable testing.
*/
protected void sysexit() {
+ // TODO: FIX THIS XTRACE
+// XTraceContext.joinParentProcess();
System.exit(0);
}
@@ -514,6 +524,7 @@ public void shutDownJob() {
// note in a workflow scenario, this may lead to creation of a new
// job (FIXME?)
// Send job-end notification
+ xtrace.log("Handling Job Finished Event");
if (getConfig().get(MRJobConfig.MR_JOB_END_NOTIFICATION_URL) != null) {
try {
LOG.info("Job end notification started for jobID : "
@@ -556,18 +567,20 @@ public void shutDownJob() {
//Bring the process down by force.
//Not needed after HADOOP-7140
LOG.info("Exiting MR AppMaster..GoodBye!");
+ xtrace.log("Application Master exiting");
sysexit();
}
private class JobFinishEventHandler implements EventHandler {
@Override
- public void handle(JobFinishEvent event) {
+ public void handle(final JobFinishEvent event) {
// Create a new thread to shutdown the AM. We should not do it in-line
// to avoid blocking the dispatcher itself.
new Thread() {
@Override
public void run() {
+ event.joinContext();
shutDownJob();
}
}.start();
@@ -798,6 +811,7 @@ protected void serviceStop() throws Exception {
@Override
public void handle(ContainerAllocatorEvent event) {
+ event.joinContext();
this.containerAllocator.handle(event);
}
@@ -849,6 +863,7 @@ protected void serviceStart() throws Exception {
@Override
public void handle(ContainerLauncherEvent event) {
+ event.joinContext();
this.containerLauncher.handle(event);
}
@@ -1181,6 +1196,7 @@ private class JobEventDispatcher implements EventHandler {
@SuppressWarnings("unchecked")
@Override
public void handle(JobEvent event) {
+ event.joinContext();
((EventHandler)context.getJob(event.getJobId())).handle(event);
}
}
@@ -1189,6 +1205,7 @@ private class TaskEventDispatcher implements EventHandler {
@SuppressWarnings("unchecked")
@Override
public void handle(TaskEvent event) {
+ event.joinContext();
Task task = context.getJob(event.getTaskID().getJobId()).getTask(
event.getTaskID());
((EventHandler)task).handle(event);
@@ -1200,6 +1217,7 @@ private class TaskAttemptEventDispatcher
@SuppressWarnings("unchecked")
@Override
public void handle(TaskAttemptEvent event) {
+ event.joinContext();
Job job = context.getJob(event.getTaskAttemptID().getTaskId().getJobId());
Task task = job.getTask(event.getTaskAttemptID().getTaskId());
TaskAttempt attempt = task.getAttempt(event.getTaskAttemptID());
@@ -1216,6 +1234,7 @@ public SpeculatorEventDispatcher(Configuration config) {
}
@Override
public void handle(SpeculatorEvent event) {
+ event.joinContext();
if (disabled) {
return;
}
@@ -1274,6 +1293,9 @@ private static void validateInputParam(String value, String param)
public static void main(String[] args) {
try {
+ // Load the XTrace context from the parent process
+ XTrace.set(System.getenv());
+ xtrace.log("Application Master Launching");
Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
String containerIdStr =
System.getenv(Environment.CONTAINER_ID.name());
@@ -1323,6 +1345,9 @@ public static void main(String[] args) {
initAndStartAppMaster(appMaster, conf, jobUserName);
} catch (Throwable t) {
LOG.fatal("Error starting MRAppMaster", t);
+ xtrace.log("Error starting MRAppMaster: "+t.getClass().getName(), "Message", t.getMessage());
+ // TODO: xtrace fix this
+// XTraceContext.joinParentProcess();
System.exit(1);
}
}
@@ -1337,6 +1362,7 @@ static class MRAppMasterShutdownHook implements Runnable {
public void run() {
LOG.info("MRAppMaster received a signal. Signaling RMCommunicator and "
+ "JobHistoryEventHandler.");
+ xtrace.log("MRAppMaster received a signal. Signaling RMCommunicator and JobHistoryEventHandler.");
// Notify the JHEH and RMCommunicator that a SIGTERM has been received so
// that they don't take too long in shutting down
@@ -1346,6 +1372,7 @@ public void run() {
}
appMaster.notifyIsLastAMRetry(appMaster.isLastAMRetry);
appMaster.stop();
+ xtrace.log("MRAppMaster stopping");
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
index 303b4c172171..bfb167436fc0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
@@ -35,6 +35,8 @@
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.Clock;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* This class keeps track of tasks that have already been launched. It
@@ -115,11 +117,12 @@ public void progressing(TaskAttemptId attemptID) {
ReportTime time = runningAttempts.get(attemptID);
if(time != null) {
time.setLastProgress(clock.getTime());
+ attemptID.rememberContext();
}
}
-
public void register(TaskAttemptId attemptID) {
+ attemptID.rememberContext();
runningAttempts.put(attemptID, new ReportTime(clock.getTime()));
}
@@ -145,12 +148,14 @@ public void run() {
if(taskTimedOut) {
// task is lost, remove from the list and raise lost event
+ entry.getKey().joinContext();
iterator.remove();
eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry
.getKey(), "AttemptID:" + entry.getKey().toString()
+ " Timed out after " + taskTimeOut / 1000 + " secs"));
eventHandler.handle(new TaskAttemptEvent(entry.getKey(),
TaskAttemptEventType.TA_TIMED_OUT));
+ XTrace.stop();
}
}
try {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java
index b14abcc6d55a..314cc7d8fdb1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java
@@ -64,6 +64,8 @@ public interface Job {
int getTotalReduces();
int getCompletedMaps();
int getCompletedReduces();
+ int getAndJoinCompletedMaps();
+ int getAndJoinCompletedReduces();
float getProgress();
boolean isUber();
String getUserName();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
index 3d54273bf22a..5c4755bef179 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
@@ -24,6 +24,7 @@
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
@@ -125,6 +126,9 @@
import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.Clock;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/** Implementation of Job interface. Maintains the state machines of Job.
* The read and write calls use ReadWriteLock for concurrency.
*/
@@ -138,6 +142,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
private static final TaskCompletionEvent[]
EMPTY_TASK_COMPLETION_EVENTS = new TaskCompletionEvent[0];
+ private static final XTrace.Logger xtrace = XTrace.getLogger(JobImpl.class);
private static final Log LOG = LogFactory.getLog(JobImpl.class);
//The maximum fraction of fetch failures allowed for a map
@@ -233,7 +238,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
StateMachineFactory
stateMachineFactory
= new StateMachineFactory
- (JobStateInternal.NEW)
+ (JobStateInternal.NEW, StateMachineFactory.Trace.KEEPALIVE)
// Transitions from NEW state
.addTransition(JobStateInternal.NEW, JobStateInternal.NEW,
@@ -595,6 +600,14 @@ JobEventType.JOB_KILL, new KillTasksTransition())
private boolean isUber = false;
private Credentials jobCredentials;
+ private Collection completedTaskContexts = new HashSet();
+ private Collection succeededMapTaskContexts = new HashSet();
+ private Collection succeededReduceTaskContexts = new HashSet();
+ private Collection failedMapTaskContexts = new HashSet();
+ private Collection failedReduceTaskContexts = new HashSet();
+ private Collection killedMapTaskContexts = new HashSet();
+ private Collection killedReduceTaskContexts = new HashSet();
+
private Token jobToken;
private JobTokenSecretManager jobTokenSecretManager;
@@ -692,6 +705,23 @@ public int getCompletedMaps() {
readLock.unlock();
}
}
+
+ @Override
+ public int getAndJoinCompletedMaps() {
+ readLock.lock();
+ try {
+ for (Context ctx : succeededMapTaskContexts)
+ XTrace.join(ctx);
+ for (Context ctx : failedMapTaskContexts)
+ XTrace.join(ctx);
+ for (Context ctx : killedMapTaskContexts)
+ XTrace.join(ctx);
+ return succeededMapTaskCount + failedMapTaskCount + killedMapTaskCount;
+ } finally {
+ readLock.unlock();
+ }
+ }
+
@Override
public int getCompletedReduces() {
@@ -703,6 +733,23 @@ public int getCompletedReduces() {
readLock.unlock();
}
}
+
+ @Override
+ public int getAndJoinCompletedReduces() {
+ readLock.lock();
+ try {
+ for (Context ctx : succeededReduceTaskContexts)
+ XTrace.join(ctx);
+ for (Context ctx : failedReduceTaskContexts)
+ XTrace.join(ctx);
+ for (Context ctx : killedReduceTaskContexts)
+ XTrace.join(ctx);
+ return succeededReduceTaskCount + failedReduceTaskCount
+ + killedReduceTaskCount;
+ } finally {
+ readLock.unlock();
+ }
+ }
@Override
public boolean isUber() {
@@ -790,6 +837,7 @@ public JobReport getReport() {
readLock.lock();
try {
JobState state = getState();
+ joinStateMachineXTraceContext();
// jobFile can be null if the job is not yet inited.
String jobFile =
@@ -911,6 +959,7 @@ protected void scheduleTasks(Set taskIDs,
* The only entry point to change the Job.
*/
public void handle(JobEvent event) {
+ event.joinContext();
if (LOG.isDebugEnabled()) {
LOG.debug("Processing " + event.getJobId() + " of type "
+ event.getType());
@@ -969,6 +1018,15 @@ private static JobState getExternalState(JobStateInternal smState) {
}
}
+ private void joinStateMachineXTraceContext() {
+ readLock.lock();
+ try {
+ getStateMachine().joinPreviousTransitionXTraceContext();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
//helpful in testing
protected void addTask(Task task) {
@@ -1012,8 +1070,10 @@ protected FileSystem getFileSystem(Configuration conf) throws IOException {
protected JobStateInternal checkReadyForCommit() {
JobStateInternal currentState = getInternalState();
+ joinStateMachineXTraceContext();
if (completedTaskCount == tasks.size()
&& currentState == JobStateInternal.RUNNING) {
+ xtrace.log("Job ready for commit.", "Completed Tasks", tasks.size(), "Current State", currentState);
eventHandler.handle(new CommitterJobCommitEvent(jobId, getJobContext()));
return JobStateInternal.COMMITTING;
}
@@ -1025,6 +1085,7 @@ JobStateInternal finished(JobStateInternal finalState) {
if (getInternalState() == JobStateInternal.RUNNING) {
metrics.endRunningJob(this);
}
+ joinStateMachineXTraceContext();
if (finishTime == 0) setFinishTime();
eventHandler.handle(new JobFinishEvent(jobId));
@@ -1368,6 +1429,7 @@ public JobStateInternal transition(JobImpl job, JobEvent event) {
// create the Tasks but don't start them yet
createMapTasks(job, inputLength, taskSplitMetaInfo);
createReduceTasks(job);
+ xtrace.log("Created map and reduce tasks");
job.metrics.endPreparingJob(job);
return JobStateInternal.INITED;
@@ -1424,7 +1486,11 @@ protected void setup(JobImpl job) throws IOException {
private void createMapTasks(JobImpl job, long inputLength,
TaskSplitMetaInfo[] splits) {
+
+ xtrace.log("Creating Map Tasks", "Input Length", inputLength, "Num Splits", splits.length);
+ Context start_context = XTrace.get();
for (int i=0; i < job.numMapTasks; ++i) {
+ XTrace.set(start_context);
TaskImpl task =
new MapTaskImpl(job.jobId, i,
job.eventHandler,
@@ -1442,7 +1508,10 @@ private void createMapTasks(JobImpl job, long inputLength,
}
private void createReduceTasks(JobImpl job) {
+ xtrace.log("Creating Reduce Tasks", "Num Reduces", job.numReduceTasks);
+ Context start_context = XTrace.get();
for (int i = 0; i < job.numReduceTasks; i++) {
+ XTrace.set(start_context);
TaskImpl task =
new ReduceTaskImpl(job.jobId, i,
job.eventHandler,
@@ -1543,6 +1612,10 @@ public void transition(JobImpl job, JobEvent event) {
private void unsuccessfulFinish(JobStateInternal finalState) {
if (finishTime == 0) setFinishTime();
cleanupProgress = 1.0f;
+ for (Context ctx : succeededMapTaskContexts)
+ XTrace.join(ctx);
+ for (Context ctx : succeededReduceTaskContexts)
+ XTrace.join(ctx);
JobUnsuccessfulCompletionEvent unsuccessfulJobEvent =
new JobUnsuccessfulCompletionEvent(oldJobId,
finishTime,
@@ -1571,6 +1644,14 @@ private static JobFinishedEvent createJobFinishedEvent(JobImpl job) {
job.mayBeConstructFinalFullCounters();
+ for (Context ctx : job.succeededMapTaskContexts)
+ XTrace.join(ctx);
+ for (Context ctx : job.succeededReduceTaskContexts)
+ XTrace.join(ctx);
+ for (Context ctx : job.failedMapTaskContexts)
+ XTrace.join(ctx);
+ for (Context ctx : job.failedReduceTaskContexts)
+ XTrace.join(ctx);
JobFinishedEvent jfe = new JobFinishedEvent(
job.oldJobId, job.finishTime,
job.succeededMapTaskCount, job.succeededReduceTaskCount,
@@ -1807,6 +1888,14 @@ protected JobStateInternal checkJobAfterTaskCompletion(JobImpl job) {
String diagnosticMsg = "Job failed as tasks failed. " +
"failedMaps:" + job.failedMapTaskCount +
" failedReduces:" + job.failedReduceTaskCount;
+
+ for (Context ctx : job.failedMapTaskContexts)
+ XTrace.join(ctx);
+ for (Context ctx : job.failedReduceTaskContexts)
+ XTrace.join(ctx);
+
+ xtrace.log("Job failed as tasks failed.", "Failed Maps", job.failedMapTaskCount, "Failed Reduces", job.failedReduceTaskCount);
+
LOG.info(diagnosticMsg);
job.addDiagnostic(diagnosticMsg);
job.eventHandler.handle(new CommitterJobAbortEvent(job.jobId,
@@ -1820,8 +1909,10 @@ protected JobStateInternal checkJobAfterTaskCompletion(JobImpl job) {
private void taskSucceeded(JobImpl job, Task task) {
if (task.getType() == TaskType.MAP) {
+ job.succeededMapTaskContexts.add(XTrace.get());
job.succeededMapTaskCount++;
} else {
+ job.succeededReduceTaskContexts.add(XTrace.get());
job.succeededReduceTaskCount++;
}
job.metrics.completedTask(task);
@@ -1829,8 +1920,10 @@ private void taskSucceeded(JobImpl job, Task task) {
private void taskFailed(JobImpl job, Task task) {
if (task.getType() == TaskType.MAP) {
+ job.failedMapTaskContexts.add(XTrace.get());
job.failedMapTaskCount++;
} else if (task.getType() == TaskType.REDUCE) {
+ job.failedReduceTaskContexts.add(XTrace.get());
job.failedReduceTaskCount++;
}
job.addDiagnostic("Task failed " + task.getID());
@@ -1839,8 +1932,10 @@ private void taskFailed(JobImpl job, Task task) {
private void taskKilled(JobImpl job, Task task) {
if (task.getType() == TaskType.MAP) {
+ job.killedMapTaskContexts.add(XTrace.get());
job.killedMapTaskCount++;
} else if (task.getType() == TaskType.REDUCE) {
+ job.killedReduceTaskContexts.add(XTrace.get());
job.killedReduceTaskCount++;
}
job.metrics.killedTask(task);
@@ -1902,6 +1997,8 @@ private static class MapTaskRescheduledTransition implements
@Override
public void transition(JobImpl job, JobEvent event) {
//succeeded map task is restarted back
+ job.succeededMapTaskContexts.add(XTrace.get());
+ job.completedTaskContexts.add(XTrace.get());
job.completedTaskCount--;
job.succeededMapTaskCount--;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index 874577d95c2f..2f92374c07bc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -196,7 +196,7 @@ public abstract class TaskAttemptImpl implements
stateMachineFactory
= new StateMachineFactory
- (TaskAttemptStateInternal.NEW)
+ (TaskAttemptStateInternal.NEW, StateMachineFactory.Trace.KEEPALIVE)
// Transitions from the NEW state.
.addTransition(TaskAttemptStateInternal.NEW, TaskAttemptStateInternal.UNASSIGNED,
@@ -255,7 +255,7 @@ TaskAttemptEventType.TA_FAILMSG, new DeallocateContainerTransition(
// Transitions from RUNNING state.
.addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.RUNNING,
- TaskAttemptEventType.TA_UPDATE, new StatusUpdater())
+ TaskAttemptEventType.TA_UPDATE, new StatusUpdater(), StateMachineFactory.Trace.IGNORE)
.addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.RUNNING,
TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
@@ -1028,6 +1028,7 @@ public TaskAttemptState getState() {
@SuppressWarnings("unchecked")
@Override
public void handle(TaskAttemptEvent event) {
+ event.joinContext();
if (LOG.isDebugEnabled()) {
LOG.debug("Processing " + event.getTaskAttemptID() + " of type "
+ event.getType());
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
index 75d833652af5..fab5e8bd82dc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
@@ -146,7 +146,7 @@ public abstract class TaskImpl implements Task, EventHandler {
stateMachineFactory
= new StateMachineFactory
- (TaskStateInternal.NEW)
+ (TaskStateInternal.NEW, StateMachineFactory.Trace.KEEPALIVE)
// define the state machine of Task
@@ -621,6 +621,7 @@ private TaskAttemptImpl addAttempt(Avataar avataar) {
@Override
public void handle(TaskEvent event) {
+ event.joinContext();
if (LOG.isDebugEnabled()) {
LOG.debug("Processing " + event.getTaskID() + " of type "
+ event.getType());
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
index 666f757b540c..05a225c4e0ec 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
@@ -58,12 +58,15 @@
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* This class is responsible for launching of containers.
*/
public class ContainerLauncherImpl extends AbstractService implements
ContainerLauncher {
+ static final XTrace.Logger xtrace = XTrace.getLogger(ContainerLauncherImpl.class);
static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
private ConcurrentHashMap containers =
@@ -124,6 +127,8 @@ public synchronized boolean isCompletelyDone() {
@SuppressWarnings("unchecked")
public synchronized void launch(ContainerRemoteLaunchEvent event) {
+ event.joinContext();
+ TaskAttemptId taskAttemptID = event.getTaskAttemptID();
LOG.info("Launching " + taskAttemptID);
if(this.state == ContainerState.KILLED_BEFORE_LAUNCH) {
state = ContainerState.DONE;
@@ -189,7 +194,6 @@ public synchronized void launch(ContainerRemoteLaunchEvent event) {
@SuppressWarnings("unchecked")
public synchronized void kill() {
-
if(this.state == ContainerState.PREP) {
this.state = ContainerState.KILLED_BEFORE_LAUNCH;
} else if (!isCompletelyDone()) {
@@ -267,6 +271,7 @@ public void run() {
Set allNodes = new HashSet();
while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
+ XTrace.stop();
try {
event = eventQueue.take();
} catch (InterruptedException e) {
@@ -276,7 +281,7 @@ public void run() {
return;
}
allNodes.add(event.getContainerMgrAddress());
-
+ event.joinContext();
int poolSize = launcherPool.getCorePoolSize();
// See if we need up the pool size only if haven't reached the
@@ -354,7 +359,10 @@ class EventProcessor implements Runnable {
@Override
public void run() {
+ XTrace.stop();
+ event.joinContext();
LOG.info("Processing the event " + event.toString());
+ xtrace.log("Processing event", "Event", event);
// Load ContainerManager tokens before creating a connection.
// TODO: Do it only once per NodeManager.
@@ -374,6 +382,7 @@ public void run() {
break;
}
removeContainerIfDone(containerID);
+ XTrace.stop();
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
index 426dc212f525..809a2e56f27d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
@@ -139,6 +139,7 @@ protected synchronized void heartbeat() throws Exception {
@SuppressWarnings("unchecked")
@Override
public void handle(ContainerAllocatorEvent event) {
+ event.joinContext();
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
LOG.info("Processing the event " + event.toString());
// Assign the same container ID as the AM
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
index 187a6e062057..b93f6fa030a4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
@@ -54,6 +54,7 @@
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import edu.brown.cs.systems.xtrace.XTrace;
/**
* Registers/unregisters to RM and sends heartbeats to RM.
*/
@@ -247,6 +248,7 @@ public void run() {
}
return;
}
+ XTrace.stop();
}
}
});
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index d6e459316323..2d9ada0ce3ad 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -75,12 +75,16 @@
import com.google.common.annotations.VisibleForTesting;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Allocates the container from the ResourceManager scheduler.
*/
public class RMContainerAllocator extends RMContainerRequestor
implements ContainerAllocator {
+ static final XTrace.Logger xtrace = XTrace.getLogger(RMContainerAllocator.class);
static final Log LOG = LogFactory.getLog(RMContainerAllocator.class);
public static final
@@ -187,6 +191,7 @@ public void run() {
ContainerAllocatorEvent event;
while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
+ XTrace.stop();
try {
event = RMContainerAllocator.this.eventQueue.take();
} catch (InterruptedException e) {
@@ -195,6 +200,7 @@ public void run() {
}
return;
}
+ event.joinContext();
try {
handleEvent(event);
@@ -221,8 +227,10 @@ protected synchronized void heartbeat() throws Exception {
scheduledRequests.assign(allocatedContainers);
}
- int completedMaps = getJob().getCompletedMaps();
- int completedTasks = completedMaps + getJob().getCompletedReduces();
+ Context start_context = XTrace.get();
+
+ int completedMaps = getJob().getAndJoinCompletedMaps();
+ int completedTasks = completedMaps + getJob().getAndJoinCompletedReduces();
if (lastCompletedTasks != completedTasks) {
lastCompletedTasks = completedTasks;
recalculateReduceSchedule = true;
@@ -239,6 +247,8 @@ protected synchronized void heartbeat() throws Exception {
maxReduceRampupLimit, reduceSlowStart);
recalculateReduceSchedule = false;
}
+
+ XTrace.set(start_context);
scheduleStats.updateAndLogIfChanged("After Scheduling: ");
}
@@ -284,8 +294,10 @@ public void handle(ContainerAllocatorEvent event) {
@SuppressWarnings({ "unchecked" })
protected synchronized void handleEvent(ContainerAllocatorEvent event) {
+ event.joinContext();
recalculateReduceSchedule = true;
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
+ xtrace.log("Processing CONTAINER_REQ Event");
ContainerRequestEvent reqEvent = (ContainerRequestEvent) event;
JobId jobId = getJob().getID();
int supportedMaxContainerCapability =
@@ -308,6 +320,8 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) {
}
}
//set the rounded off memory
+ xtrace.log("Scheduling Map Container Request");
+ reqEvent.rememberContext();
reqEvent.getCapability().setMemory(mapResourceReqt);
scheduledRequests.addMap(reqEvent);//maps are immediately scheduled
} else {
@@ -333,8 +347,12 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) {
reqEvent.getCapability().setMemory(reduceResourceReqt);
if (reqEvent.getEarlierAttemptFailed()) {
//add to the front of queue for fail fast
+ xtrace.log("Scheduling Fail-Fast Reduce Container Request");
+ reqEvent.rememberContext();
pendingReduces.addFirst(new ContainerRequest(reqEvent, PRIORITY_REDUCE));
} else {
+ xtrace.log("Scheduling Reduce Container Request");
+ reqEvent.rememberContext();
pendingReduces.add(new ContainerRequest(reqEvent, PRIORITY_REDUCE));
//reduces are added to pending and are slowly ramped up
}
@@ -342,6 +360,7 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) {
} else if (
event.getType() == ContainerAllocator.EventType.CONTAINER_DEALLOCATE) {
+ xtrace.log("Processing CONTAINER_DEALLOCATE Event");
LOG.info("Processing the event " + event.toString());
@@ -363,6 +382,7 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) {
}
} else if (
event.getType() == ContainerAllocator.EventType.CONTAINER_FAILED) {
+ xtrace.log("Processing CONTAINER_FAILED Event");
ContainerFailedEvent fEv = (ContainerFailedEvent) event;
String host = getHost(fEv.getContMgrAddress());
containerFailedOnHost(host);
@@ -515,18 +535,29 @@ public void scheduleReduces(
@Private
public void scheduleAllReduces() {
+ xtrace.log("Scheduling all reduces");
+ Context start_context = XTrace.get();
for (ContainerRequest req : pendingReduces) {
+ req.joinContext();
+ xtrace.log("Scheduling reduce");
+ req.rememberContext();
scheduledRequests.addReduce(req);
+ XTrace.set(start_context);
}
pendingReduces.clear();
}
@Private
public void rampUpReduces(int rampUp) {
- //more reduce to be scheduled
+ xtrace.log("Ramping up reduces", "Ramp Up", rampUp);
+ Context start_context = XTrace.get();
for (int i = 0; i < rampUp; i++) {
ContainerRequest request = pendingReduces.removeFirst();
+ request.joinContext();
+ xtrace.log("Request ramping up");
+ request.rememberContext();
scheduledRequests.addReduce(request);
+ XTrace.set(start_context);
}
}
@@ -737,6 +768,8 @@ ContainerRequest removeReduce() {
}
void addMap(ContainerRequestEvent event) {
+ event.joinContext();
+
ContainerRequest request = null;
if (event.getEarlierAttemptFailed()) {
@@ -783,8 +816,10 @@ private void assign(List allocatedContainers) {
Iterator it = allocatedContainers.iterator();
LOG.info("Got allocated containers " + allocatedContainers.size());
containersAllocated += allocatedContainers.size();
+
while (it.hasNext()) {
Container allocated = it.next();
+ allocated.getId().joinContext();
if (LOG.isDebugEnabled()) {
LOG.debug("Assigning container " + allocated.getId()
+ " with priority " + allocated.getPriority() + " to NM "
@@ -832,6 +867,7 @@ else if (PRIORITY_REDUCE.equals(priority)) {
continue;
}
+ ContainerId allocatedContainerId = allocated.getId();
// do not assign if allocated container is on a
// blacklisted host
String allocatedHost = allocated.getNodeId().getHost();
@@ -871,6 +907,8 @@ else if (PRIORITY_REDUCE.equals(priority)) {
it.remove();
continue;
}
+
+ XTrace.stop();
}
assignContainers(allocatedContainers);
@@ -879,15 +917,21 @@ else if (PRIORITY_REDUCE.equals(priority)) {
it = allocatedContainers.iterator();
while (it.hasNext()) {
Container allocated = it.next();
+ allocated.getId().joinContext();
LOG.info("Releasing unassigned and invalid container "
+ allocated + ". RM may have assignment issues");
containerNotAssigned(allocated);
+ XTrace.stop();
}
+
+ XTrace.stop();
}
@SuppressWarnings("unchecked")
private void containerAssigned(Container allocated,
ContainerRequest assigned) {
+ xtrace.log("Container Assigned", "Container ID", allocated.getId(), "Assigned ID", assigned.attemptID);
+
// Update resource requests
decContainerReq(assigned);
@@ -916,11 +960,15 @@ private ContainerRequest assignWithoutLocality(Container allocated) {
if (PRIORITY_FAST_FAIL_MAP.equals(priority)) {
LOG.info("Assigning container " + allocated + " to fast fail map");
assigned = assignToFailedMap(allocated);
+ allocated.getId().joinContext();
+ xtrace.log("Assigned container for fast fail map", "Container ID", allocated.getId(), "Assigned ID", assigned.attemptID);
} else if (PRIORITY_REDUCE.equals(priority)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Assigning container " + allocated + " to reduce");
}
assigned = assignToReduce(allocated);
+ allocated.getId().joinContext();
+ xtrace.log("Assigned container to reduce", "Container ID", allocated.getId(), "Assigned ID", assigned.attemptID);
}
return assigned;
@@ -929,6 +977,7 @@ private ContainerRequest assignWithoutLocality(Container allocated) {
private void assignContainers(List allocatedContainers) {
Iterator it = allocatedContainers.iterator();
while (it.hasNext()) {
+ XTrace.stop();
Container allocated = it.next();
ContainerRequest assigned = assignWithoutLocality(allocated);
if (assigned != null) {
@@ -936,8 +985,10 @@ private void assignContainers(List allocatedContainers) {
it.remove();
}
}
+ XTrace.stop();
assignMapsWithLocality(allocatedContainers);
+ XTrace.stop();
}
private ContainerRequest getContainerReqToReplace(Container allocated) {
@@ -986,6 +1037,7 @@ private ContainerRequest assignToFailedMap(Container allocated) {
TaskAttemptId tId = earlierFailedMaps.removeFirst();
if (maps.containsKey(tId)) {
assigned = maps.remove(tId);
+ assigned.joinContext();
JobCounterUpdateEvent jce =
new JobCounterUpdateEvent(assigned.attemptID.getTaskId().getJobId());
jce.addCounterUpdate(JobCounter.OTHER_LOCAL_MAPS, 1);
@@ -1003,6 +1055,7 @@ private ContainerRequest assignToReduce(Container allocated) {
if (assigned == null && reduces.size() > 0) {
TaskAttemptId tId = reduces.keySet().iterator().next();
assigned = reduces.remove(tId);
+ assigned.joinContext();
LOG.info("Assigned to reduce");
}
return assigned;
@@ -1013,13 +1066,16 @@ private void assignMapsWithLocality(List allocatedContainers) {
// try to assign to all nodes first to match node local
Iterator it = allocatedContainers.iterator();
while(it.hasNext() && maps.size() > 0){
- Container allocated = it.next();
+ Container allocated = it.next();
+ XTrace.stop();
+ allocated.getId().joinContext();
Priority priority = allocated.getPriority();
assert PRIORITY_MAP.equals(priority);
// "if (maps.containsKey(tId))" below should be almost always true.
// hence this while loop would almost always have O(1) complexity
String host = allocated.getNodeId().getHost();
LinkedList list = mapsHostMapping.get(host);
+
while (list != null && list.size() > 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("Host matched to the request list " + host);
@@ -1027,6 +1083,8 @@ private void assignMapsWithLocality(List allocatedContainers) {
TaskAttemptId tId = list.removeFirst();
if (maps.containsKey(tId)) {
ContainerRequest assigned = maps.remove(tId);
+ assigned.joinContext();
+ xtrace.log("Assigned container based on host match", "Host", host);
containerAssigned(allocated, assigned);
it.remove();
JobCounterUpdateEvent jce =
@@ -1041,11 +1099,14 @@ private void assignMapsWithLocality(List allocatedContainers) {
}
}
}
+ XTrace.stop();
// try to match all rack local
it = allocatedContainers.iterator();
while(it.hasNext() && maps.size() > 0){
+ XTrace.stop();
Container allocated = it.next();
+ allocated.getId().joinContext();
Priority priority = allocated.getPriority();
assert PRIORITY_MAP.equals(priority);
// "if (maps.containsKey(tId))" below should be almost always true.
@@ -1057,6 +1118,8 @@ private void assignMapsWithLocality(List allocatedContainers) {
TaskAttemptId tId = list.removeFirst();
if (maps.containsKey(tId)) {
ContainerRequest assigned = maps.remove(tId);
+ assigned.joinContext();
+ xtrace.log("Assigned container based on rack match", "Rack", rack);
containerAssigned(allocated, assigned);
it.remove();
JobCounterUpdateEvent jce =
@@ -1069,17 +1132,23 @@ private void assignMapsWithLocality(List allocatedContainers) {
}
break;
}
+ XTrace.stop();
}
}
+ XTrace.stop();
// assign remaining
it = allocatedContainers.iterator();
while(it.hasNext() && maps.size() > 0){
+ XTrace.stop();
Container allocated = it.next();
+ allocated.getId().joinContext();
Priority priority = allocated.getPriority();
assert PRIORITY_MAP.equals(priority);
TaskAttemptId tId = maps.keySet().iterator().next();
ContainerRequest assigned = maps.remove(tId);
+ assigned.joinContext();
+ xtrace.log("Assigned container based on * match");
containerAssigned(allocated, assigned);
it.remove();
JobCounterUpdateEvent jce =
@@ -1089,7 +1158,9 @@ private void assignMapsWithLocality(List allocatedContainers) {
if (LOG.isDebugEnabled()) {
LOG.debug("Assigned based on * match");
}
+ XTrace.stop();
}
+ XTrace.stop();
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
index 67dd30e16418..90ee325c5fee 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
@@ -38,7 +38,9 @@
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -47,12 +49,14 @@
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import edu.brown.cs.systems.xtrace.XTrace;
/**
* Keeps the data structures to send container requests to RM.
*/
public abstract class RMContainerRequestor extends RMCommunicator {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(RMContainerRequestor.class);
private static final Log LOG = LogFactory.getLog(RMContainerRequestor.class);
private int lastResponseID;
@@ -98,10 +102,12 @@ static class ContainerRequest {
final String[] racks;
//final boolean earlierAttemptFailed;
final Priority priority;
+ private ContainerRequestEvent event;
public ContainerRequest(ContainerRequestEvent event, Priority priority) {
this(event.getAttemptID(), event.getCapability(), event.getHosts(),
event.getRacks(), priority);
+ this.event = event;
}
public ContainerRequest(TaskAttemptId attemptID,
@@ -121,6 +127,19 @@ public String toString() {
sb.append("Priority[").append(priority).append("]");
return sb.toString();
}
+
+ public void joinContext() {
+ if (event!=null) {
+ event.joinContext();
+ }
+ }
+
+ public void rememberContext() {
+ if (event!=null) {
+ event.rememberContext();
+ }
+ }
+
}
@Override
@@ -145,6 +164,23 @@ protected void serviceInit(Configuration conf) throws Exception {
}
protected AllocateResponse makeRemoteRequest() throws IOException {
+
+ for (ResourceRequest r : ask) {
+ XTrace.stop();
+ r.joinContext();
+ xtrace.log("Requesting container from RM");
+ r.rememberContext();
+ }
+
+ for (ContainerId i : release) {
+ XTrace.stop();
+ i.joinContext();
+ xtrace.log("Requesting RM release container", "Container ID", i);
+ i.rememberContext();
+ }
+ XTrace.stop();
+
+
AllocateRequest allocateRequest =
AllocateRequest.newInstance(lastResponseID,
super.getApplicationProgress(), new ArrayList(ask),
@@ -169,6 +205,22 @@ protected AllocateResponse makeRemoteRequest() throws IOException {
+ " resourcelimit=" + availableResources + " knownNMs="
+ clusterNmCount);
}
+
+ for (Container x : allocateResponse.getAllocatedContainers()) {
+ XTrace.stop();
+ x.getId().joinContext();
+ xtrace.log("Container allocated by RM");
+ x.getId().rememberContext();
+ }
+
+ for (ContainerStatus x : allocateResponse.getCompletedContainersStatuses()) {
+ XTrace.stop();
+ x.getContainerId().joinContext();
+ xtrace.log("RM acknowledged completed container");
+ x.getContainerId().rememberContext();
+
+ }
+ XTrace.stop();
ask.clear();
release.clear();
@@ -325,7 +377,14 @@ private void addResourceRequest(Priority priority, String resourceName,
remoteRequest.setResourceName(resourceName);
remoteRequest.setCapability(capability);
remoteRequest.setNumContainers(0);
+// remoteRequest.rememberContext();
reqMap.put(capability, remoteRequest);
+ } else {
+ // XTrace - if multiple people request containers at once, this creates unwanted cross-products
+// Collection start_context = XTraceContext.getThreadContext();
+// remoteRequest.joinContext();
+// remoteRequest.rememberContext();
+// XTraceContext.setThreadContext(start_context);
}
remoteRequest.setNumContainers(remoteRequest.getNumContainers() + 1);
@@ -368,7 +427,6 @@ private void decResourceRequest(Priority priority, String resourceName,
// than requested. so guard for that.
remoteRequest.setNumContainers(remoteRequest.getNumContainers() -1);
}
-
if (remoteRequest.getNumContainers() == 0) {
reqMap.remove(capability);
if (reqMap.size() == 0) {
@@ -398,10 +456,12 @@ private void addResourceRequestToAsk(ResourceRequest remoteRequest) {
if(ask.contains(remoteRequest)) {
ask.remove(remoteRequest);
}
+ remoteRequest.rememberContext();
ask.add(remoteRequest);
}
protected void release(ContainerId containerId) {
+ containerId.rememberContext();
release.add(containerId);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java
index 532a9a2ee73b..654b12e49dd1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java
@@ -52,7 +52,6 @@
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.util.Clock;
-
public class DefaultSpeculator extends AbstractService implements
Speculator {
@@ -428,6 +427,7 @@ protected void addSpeculativeAttempt(TaskId taskID) {
@Override
public void handle(SpeculatorEvent event) {
+ event.joinContext();
processSpeculatorEvent(event);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
index 7073ba4844e9..d52d3e2e5f50 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
@@ -466,6 +466,7 @@ public MockContainerLauncher() {
@Override
public void handle(ContainerLauncherEvent event) {
+ event.joinContext();
switch (event.getType()) {
case CONTAINER_REMOTE_LAUNCH:
getContext().getEventHandler().handle(
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
index 0a74ebdd01f7..5e4a18b594c9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
@@ -615,6 +615,16 @@ public Configuration loadConfFile() throws IOException {
jobConf.addResource(fc.open(configFile), configFile.toString());
return jobConf;
}
+
+ @Override
+ public int getAndJoinCompletedMaps() {
+ return getCompletedMaps();
+ }
+
+ @Override
+ public int getAndJoinCompletedReduces() {
+ return getCompletedReduces();
+ }
};
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
index 762dd572f3a8..6cdb2058d924 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
@@ -505,6 +505,16 @@ public List getAMInfos() {
public Configuration loadConfFile() {
throw new UnsupportedOperationException();
}
+
+ @Override
+ public int getAndJoinCompletedMaps() {
+ return getCompletedMaps();
+ }
+
+ @Override
+ public int getAndJoinCompletedReduces() {
+ return getCompletedReduces();
+ }
}
/*
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
index d71aff518d03..bc360328c93d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
@@ -68,12 +68,15 @@
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/** Implements MapReduce locally, in-process, for debugging. */
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class LocalJobRunner implements ClientProtocol {
- public static final Log LOG =
- LogFactory.getLog(LocalJobRunner.class);
+ public static final XTrace.Logger xtrace = XTrace.getLogger(LocalJobRunner.class);
+ public static final Log LOG = LogFactory.getLog(LocalJobRunner.class);
/** The maximum number of map tasks to run in parallel in LocalJobRunner */
public static final String LOCAL_MAX_MAPS =
@@ -128,6 +131,7 @@ private class Job extends Thread implements TaskUmbilicalProtocol {
boolean killed = false;
private LocalDistributedCacheManager localDistributedCacheManager;
+ private final Context xtrace_context;
public long getProtocolVersion(String protocol, long clientVersion) {
return TaskUmbilicalProtocol.versionID;
@@ -178,6 +182,8 @@ public Job(JobID jobid, String jobSubmitDir) throws IOException {
profile.getURL().toString());
jobs.put(id, this);
+
+ this.xtrace_context = XTrace.get();
this.start();
}
@@ -197,6 +203,7 @@ protected class MapTaskRunnable implements Runnable {
private final Map mapOutputFiles;
public volatile Throwable storedException;
+ private Context runnable_end_context;
public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId,
Map mapOutputFiles) {
@@ -240,6 +247,8 @@ public void run() {
} catch (Throwable e) {
this.storedException = e;
}
+ this.runnable_end_context = XTrace.get();
+ XTrace.stop();
}
}
@@ -342,6 +351,8 @@ protected ExecutorService createMapExecutor(int numMapTasks) {
@Override
public void run() {
+ XTrace.set(xtrace_context);
+
JobID jobId = profile.getJobID();
JobContext jContext = new JobContextImpl(job, jobId);
@@ -391,7 +402,13 @@ public void run() {
mapService.shutdownNow();
throw ie;
}
-
+
+ XTrace.stop();
+ for (MapTaskRunnable r : taskRunnables) {
+ XTrace.join(r.runnable_end_context);
+ }
+
+ xtrace.log("Map tasks complete");
LOG.info("Map task executor complete.");
// After waiting for the map tasks to complete, if any of these
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskAttemptId.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskAttemptId.java
index bae90991da51..3c328d10aed0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskAttemptId.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskAttemptId.java
@@ -18,6 +18,9 @@
package org.apache.hadoop.mapreduce.v2.api.records;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
*
* TaskAttemptId
represents the unique identifier for a task
@@ -32,6 +35,17 @@
*
*/
public abstract class TaskAttemptId implements Comparable {
+
+ private Context xtrace_context;
+
+ public void rememberContext() {
+ xtrace_context = XTrace.get();
+ }
+
+ public void joinContext() {
+ XTrace.join(xtrace_context);
+ }
+
/**
* @return the associated TaskId.
*/
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/CleanupQueue.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/CleanupQueue.java
index 456ed7c689cf..89c6d36cbb72 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/CleanupQueue.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/CleanupQueue.java
@@ -23,7 +23,6 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IFile.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IFile.java
index a410c975578f..9403e7d6e9ab 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IFile.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IFile.java
@@ -24,6 +24,8 @@
import java.io.IOException;
import java.io.InputStream;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@@ -43,8 +45,8 @@
import org.apache.hadoop.io.serializer.SerializationFactory;
import org.apache.hadoop.io.serializer.Serializer;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import edu.brown.cs.systems.resourcethrottling.LocalThrottlingPoints;
+import edu.brown.cs.systems.resourcethrottling.ThrottlingPoint;
/**
* IFile
is the simple format
@@ -58,6 +60,8 @@
public class IFile {
private static final Log LOG = LogFactory.getLog(IFile.class);
public static final int EOF_MARKER = -1; // End of File Marker
+
+ private static ThrottlingPoint throttler = LocalThrottlingPoints.getThrottlingPoint("IFile");
/**
* IFile.Writer
to write out intermediate map-outputs.
@@ -228,6 +232,8 @@ public void append(K key, V value) throws IOException {
WritableUtils.getVIntSize(keyLength) +
WritableUtils.getVIntSize(valueLength);
++numRecordsWritten;
+
+ throttler.throttle();
}
public void append(DataInputBuffer key, DataInputBuffer value)
@@ -254,6 +260,8 @@ public void append(DataInputBuffer key, DataInputBuffer value)
WritableUtils.getVIntSize(keyLength) +
WritableUtils.getVIntSize(valueLength);
++numRecordsWritten;
+
+ throttler.throttle();
}
// Required for mark/reset
@@ -378,6 +386,8 @@ public long getPosition() throws IOException {
* @throws IOException
*/
private int readData(byte[] buf, int off, int len) throws IOException {
+ throttler.throttle();
+
int bytesRead = 0;
while (bytesRead < len) {
int n = IOUtils.wrappedReadForCompressedData(in, buf, off + bytesRead,
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IndexRecord.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IndexRecord.java
index 3996534bd54e..f6bc7d15e1ad 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IndexRecord.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/IndexRecord.java
@@ -20,18 +20,69 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.Metadata.XTraceMetadata;
+import edu.brown.cs.systems.xtrace.Metadata.XTraceMetadata.Builder;
+import edu.brown.cs.systems.xtrace.XTrace;
+
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
public class IndexRecord {
public long startOffset;
public long rawLength;
public long partLength;
+ public Context ctx;
public IndexRecord() { }
- public IndexRecord(long startOffset, long rawLength, long partLength) {
+ public IndexRecord(long startOffset, long rawLength, long partLength, long xtrace_taskid, long xtrace_opid) {
this.startOffset = startOffset;
this.rawLength = rawLength;
this.partLength = partLength;
+ if (xtrace_taskid!=0) {
+ Builder builder = XTraceMetadata.newBuilder().setTaskID(xtrace_taskid);
+ if (xtrace_opid != 0)
+ builder.addParentEventID(xtrace_opid);
+ ctx = Context.parse(builder.build().toByteArray());
+ }
+ }
+
+ public long getXTraceTaskID() {
+ if (ctx!=null) {
+ try {
+ return XTraceMetadata.parseFrom(ctx.bytes()).getTaskID();
+ } catch (Exception e) {
+ }
+ }
+ return 0L;
+ }
+
+ public long getXTraceOpID() {
+ if (ctx!=null) {
+ try {
+ XTraceMetadata md = XTraceMetadata.parseFrom(ctx.bytes());
+ if (md.getParentEventIDCount() > 0)
+ return md.getParentEventID(0);
+ } catch (Exception e) {
+ }
+ }
+ return 0L;
+ }
+
+ public void rememberContext() {
+ ctx = XTrace.get();
+ }
+
+ public void clearContext() {
+ ctx = null;
+ }
+
+ public void joinContext() {
+ XTrace.join(ctx);
+ }
+
+ public boolean hasContext() {
+ return ctx!=null;
}
+
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobEndNotifier.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobEndNotifier.java
index 74861fece3bf..cb3b83381065 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobEndNotifier.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobEndNotifier.java
@@ -37,8 +37,8 @@ public class JobEndNotifier {
private static final Log LOG =
LogFactory.getLog(JobEndNotifier.class.getName());
-
-
+
+
private static JobEndStatusInfo createNotification(JobConf conf,
JobStatus status) {
JobEndStatusInfo notification = null;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
index 84404d1ef13d..fe9dd8c3ce67 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
@@ -28,6 +28,8 @@
import java.nio.ByteOrder;
import java.nio.IntBuffer;
import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
import java.util.List;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
@@ -73,6 +75,8 @@
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.util.StringUtils;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/** A Map task. */
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
@@ -80,11 +84,12 @@ public class MapTask extends Task {
/**
* The size of each record in the index file for the map-outputs.
*/
- public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
+ public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 40; // xtrace: 16 bytes added for XTrace metadata
private TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
private final static int APPROX_HEADER_LENGTH = 150;
+ private static final XTrace.Logger xtrace = XTrace.getLogger(MapTask.class);
private static final Log LOG = LogFactory.getLog(MapTask.class.getName());
private Progress mapPhase;
@@ -258,6 +263,9 @@ public synchronized boolean next(K key, V value)
boolean ret = moveToNext(key, value);
long nextRecIndex = skipIt.next();
long skip = 0;
+ if (recIndex 0) {
setPhase(TaskStatus.Phase.SORT);
@@ -444,6 +458,7 @@ void runOldMapper(final JobConf job,
closeQuietly(in);
closeQuietly(collector);
}
+ xtrace.log("OldMapper Sort phase end");
}
/**
@@ -730,6 +745,7 @@ void runNewMapper(final JobConf job,
split = getSplitDetails(new Path(splitIndex.getSplitLocation()),
splitIndex.getStartOffset());
LOG.info("Processing split: " + split);
+ xtrace.log("NewMapper Processing split", "Split", split);
org.apache.hadoop.mapreduce.RecordReader input =
new NewTrackingRecordReader
@@ -760,14 +776,18 @@ void runNewMapper(final JobConf job,
try {
input.initialize(split, mapperContext);
+ xtrace.log("NewMapper Map start");
mapper.run(mapperContext);
mapPhase.complete();
+ xtrace.log("NewMapper Map end");
+ xtrace.log("NewMapper Sort phase start");
setPhase(TaskStatus.Phase.SORT);
statusUpdate(umbilical);
input.close();
input = null;
output.close(mapperContext);
output = null;
+ xtrace.log("NewMapper Sort phase end");
} finally {
closeQuietly(input);
closeQuietly(output, mapperContext);
@@ -1087,6 +1107,7 @@ public synchronized void collect(K key, V value, final int partition
} else if (bufsoftlimit && kvindex != kvend) {
// spill records, if any collected; check latter, as it may
// be possible for metadata alignment to hit spill pcnt
+ xtrace.log("Triggering asynchronous spill");
startSpill();
final int avgRec = (int)
(mapOutputByteCounter.getCounter() /
@@ -1158,6 +1179,7 @@ public synchronized void collect(K key, V value, final int partition
kvindex = (kvindex - NMETA + kvmeta.capacity()) % kvmeta.capacity();
} catch (MapBufferTooSmallException e) {
LOG.info("Record too large for in-memory buffer: " + e.getMessage());
+ xtrace.log("Record too large for in-memory buffer", "Message", e.getMessage());
spillSingleRecord(key, value, partition);
mapOutputRecordCounter.increment(1);
return;
@@ -1390,6 +1412,7 @@ public void write(byte b[], int off, int len)
}
// we have records we can spill; only spill if blocked
if (kvindex != kvend) {
+ xtrace.log("Triggering synchronous spill");
startSpill();
// Blocked on this write, waiting for the spill just
// initiated to finish. Instead of repositioning the marker
@@ -1417,6 +1440,7 @@ public void write(byte b[], int off, int len)
reporter.progress();
spillDone.await();
}
+ spillThread.joinSpillDoneContext();
} catch (InterruptedException e) {
throw new IOException(
"Buffer interrupted while waiting for the writer", e);
@@ -1443,6 +1467,7 @@ public void write(byte b[], int off, int len)
public void flush() throws IOException, ClassNotFoundException,
InterruptedException {
LOG.info("Starting flush of map output");
+ xtrace.log("Starting flush of map output");
spillLock.lock();
try {
while (spillInProgress) {
@@ -1500,16 +1525,40 @@ public void close() { }
protected class SpillThread extends Thread {
+ private edu.brown.cs.systems.xtrace.Context xtrace_spillstart_context;
+ private edu.brown.cs.systems.xtrace.Context xtrace_spilldone_context;
+
+ public void rememberSpillStartContext() {
+ xtrace_spillstart_context = XTrace.get();
+ }
+
+ public void rememberSpillDoneContext() {
+ xtrace_spilldone_context = XTrace.get();
+ }
+
+ public void joinSpillDoneContext() {
+ XTrace.join(xtrace_spilldone_context);
+ xtrace_spilldone_context = null;
+ }
+
+ public void joinSpillStartContext() {
+ XTrace.join(xtrace_spillstart_context);
+ xtrace_spillstart_context = null;
+ }
+
@Override
public void run() {
spillLock.lock();
spillThreadRunning = true;
try {
while (true) {
+ XTrace.stop();
spillDone.signal();
while (!spillInProgress) {
spillReady.await();
}
+ joinSpillStartContext();
+ xtrace.log("Spill Thread notified");
try {
spillLock.unlock();
sortAndSpill();
@@ -1524,6 +1573,7 @@ public void run() {
bufstart = bufend;
spillInProgress = false;
}
+ rememberSpillDoneContext();
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
@@ -1560,11 +1610,15 @@ private void startSpill() {
"); length = " + (distanceTo(kvend, kvstart,
kvmeta.capacity()) + 1) + "/" + maxRec);
}
+ spillThread.rememberSpillStartContext();
spillReady.signal();
}
private void sortAndSpill() throws IOException, ClassNotFoundException,
InterruptedException {
+ spillThread.joinSpillDoneContext(); // join up with the previous 'done' context if it hasn't already been joined up with. don't want it dangling
+ xtrace.log("Beginning spill", "Spill Number", numSpills);
+
//approximate the length of the output file to be the length of the
//buffer + header lengths for the partitions
final long size = (bufend >= bufstart
@@ -1584,11 +1638,19 @@ private void sortAndSpill() throws IOException, ClassNotFoundException,
(kvstart >= kvend
? kvstart
: kvmeta.capacity() + kvstart) / NMETA;
+ xtrace.log("Sorting buffer contents");
sorter.sort(MapOutputBuffer.this, mstart, mend, reporter);
int spindex = mstart;
final IndexRecord rec = new IndexRecord();
final InMemValBytes value = new InMemValBytes();
+
+
+ xtrace.log("Spilling partitions to disk", "NumPartitions", partitions, "File", filename);
+ edu.brown.cs.systems.xtrace.Context start_context = XTrace.get();
+ Collection end_contexts = new HashSet();
for (int i = 0; i < partitions; ++i) {
+ XTrace.set(start_context);
+ xtrace.log("Spilling partition", "Partition Number", i);
IFile.Writer writer = null;
try {
long segmentStart = out.getPos();
@@ -1597,6 +1659,7 @@ private void sortAndSpill() throws IOException, ClassNotFoundException,
if (combinerRunner == null) {
// spill directly
DataInputBuffer key = new DataInputBuffer();
+ int spillcount = 0;
while (spindex < mend &&
kvmeta.get(offsetFor(spindex % maxRec) + PARTITION) == i) {
final int kvoff = offsetFor(spindex % maxRec);
@@ -1606,6 +1669,14 @@ private void sortAndSpill() throws IOException, ClassNotFoundException,
getVBytesForOffset(kvoff, value);
writer.append(key, value);
++spindex;
+ ++spillcount;
+ }
+ if (spillcount > 0) {
+ xtrace.log("Spilled records directly without combine", "Partition", i, "SpillCount", spillcount);
+ rec.rememberContext();
+ } else {
+ xtrace.log("No records to spill", "Partition", i);
+ rec.clearContext();
}
} else {
int spstart = spindex;
@@ -1621,6 +1692,11 @@ private void sortAndSpill() throws IOException, ClassNotFoundException,
RawKeyValueIterator kvIter =
new MRResultIterator(spstart, spindex);
combinerRunner.combine(kvIter, combineCollector);
+ xtrace.log("Spilled records with combine", "Partition", i, "SpillCount", (spindex-spstart));
+ rec.rememberContext();
+ } else {
+ xtrace.log("No records to spill", "Partition", i);
+ rec.clearContext();
}
}
@@ -1636,8 +1712,11 @@ private void sortAndSpill() throws IOException, ClassNotFoundException,
writer = null;
} finally {
if (null != writer) writer.close();
+ end_contexts.add(XTrace.get());
}
}
+ for (edu.brown.cs.systems.xtrace.Context ctx : end_contexts)
+ XTrace.join(ctx);
if (totalIndexCacheMemory >= indexCacheMemoryLimit) {
// create spill index file
@@ -1651,6 +1730,7 @@ private void sortAndSpill() throws IOException, ClassNotFoundException,
spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
}
LOG.info("Finished spill " + numSpills);
+ xtrace.log("Finished spill", "Spill Number", numSpills);
++numSpills;
} finally {
if (out != null) out.close();
@@ -1689,6 +1769,8 @@ private void spillSingleRecord(final K key, final V value,
// Note that our map byte count will not be accurate with
// compression
mapOutputByteCounter.increment(out.getPos() - recordStart);
+ xtrace.log("Spilled single record", "Partition", partition);
+ rec.rememberContext();
}
writer.close();
@@ -1794,6 +1876,7 @@ public void close() { }
private void mergeParts() throws IOException, InterruptedException,
ClassNotFoundException {
+
// get the approximate size of the final output/index files
long finalOutFileSize = 0;
long finalIndexFileSize = 0;
@@ -1804,26 +1887,41 @@ private void mergeParts() throws IOException, InterruptedException,
filename[i] = mapOutputFile.getSpillFile(i);
finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
}
+
+ // read in paged indices
+ for (int i = indexCacheList.size(); i < numSpills; ++i) {
+ Path indexFileName = mapOutputFile.getSpillIndexFile(i);
+ indexCacheList.add(new SpillRecord(indexFileName, job));
+ }
+
if (numSpills == 1) { //the spill is the final output
+ xtrace.log("Single spill, merge unnecessary");
+
sameVolRename(filename[0],
mapOutputFile.getOutputFileForWriteInVolume(filename[0]));
if (indexCacheList.size() == 0) {
sameVolRename(mapOutputFile.getSpillIndexFile(0),
mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]));
- } else {
- indexCacheList.get(0).writeToFile(
+ } else {
+ SpillRecord spillRec = indexCacheList.get(0);
+ edu.brown.cs.systems.xtrace.Context start_context = XTrace.get();
+ for (int i = 0; i < partitions; i++) {
+ IndexRecord rec = spillRec.getIndex(i);
+ if (rec.hasContext()) {
+ rec.joinContext();
+ xtrace.log("Spill file containing partition was renamed");
+ rec.rememberContext();
+ spillRec.putIndex(rec, i);
+ XTrace.set(start_context);
+ }
+ }
+ spillRec.writeToFile(
mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]), job);
}
sortPhase.complete();
return;
}
- // read in paged indices
- for (int i = indexCacheList.size(); i < numSpills; ++i) {
- Path indexFileName = mapOutputFile.getSpillIndexFile(i);
- indexCacheList.add(new SpillRecord(indexFileName, job));
- }
-
//make correction in the length to include the sequence file header
//lengths for each partition
finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
@@ -1864,12 +1962,16 @@ private void mergeParts() throws IOException, InterruptedException,
IndexRecord rec = new IndexRecord();
final SpillRecord spillRec = new SpillRecord(partitions);
+ edu.brown.cs.systems.xtrace.Context start_context = XTrace.get();
+ Collection end_contexts = new HashSet();
for (int parts = 0; parts < partitions; parts++) {
+ XTrace.set(start_context);
//create the segments to be merged
List> segmentList =
new ArrayList>(numSpills);
for(int i = 0; i < numSpills; i++) {
IndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
+ indexRecord.joinContext();
Segment s =
new Segment(job, rfs, filename[i], indexRecord.startOffset,
@@ -1887,6 +1989,7 @@ private void mergeParts() throws IOException, InterruptedException,
// sort the segments only if there are intermediate merges
boolean sortSegments = segmentList.size() > mergeFactor;
//merge
+ xtrace.log("Merging parts from multiple spills", "Partition", parts, "Num Spills", numSpills);
@SuppressWarnings("unchecked")
RawKeyValueIterator kvIter = Merger.merge(job, rfs,
keyClass, valClass, codec,
@@ -1901,6 +2004,9 @@ private void mergeParts() throws IOException, InterruptedException,
new Writer(job, finalOut, keyClass, valClass, codec,
spilledRecordsCounter);
if (combinerRunner == null || numSpills < minSpillsForCombine) {
+ xtrace.log("Skipping combine",
+ "CombinerRunnerIsNull", combinerRunner==null, "MinSpillsForCombine", minSpillsForCombine,
+ "Partition", parts, "Num Spills", numSpills);
Merger.writeFile(kvIter, writer, reporter, job);
} else {
combineCollector.setWriter(writer);
@@ -1916,13 +2022,19 @@ private void mergeParts() throws IOException, InterruptedException,
rec.startOffset = segmentStart;
rec.rawLength = writer.getRawLength();
rec.partLength = writer.getCompressedLength();
+ rec.rememberContext();
spillRec.putIndex(rec, parts);
+
+ end_contexts.add(XTrace.get());
}
+ for (edu.brown.cs.systems.xtrace.Context ctx : end_contexts)
+ XTrace.join(ctx);
spillRec.writeToFile(finalIndexFile, job);
finalOut.close();
for(int i = 0; i < numSpills; i++) {
rfs.delete(filename[i],true);
}
+ xtrace.log("Final output written", "Final Index File", finalIndexFile.toString(), "Final Output File", finalOutputFile.toString());
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java
index ced9040f4136..96ab3b689780 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java
@@ -28,8 +28,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.ChecksumFileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
@@ -43,6 +43,8 @@
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.Progressable;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Merger is an utility class used by the Map and Reduce tasks for merging
* both their memory and disk segments
@@ -50,6 +52,7 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class Merger {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(Merger.class);
private static final Log LOG = LogFactory.getLog(Merger.class);
// Local directories
@@ -566,6 +569,7 @@ RawKeyValueIterator merge(Class keyClass, Class valueClass,
Progress mergePhase)
throws IOException {
LOG.info("Merging " + segments.size() + " sorted segments");
+ xtrace.log("Merging sorted segments", "Num Segments", + segments.size());
/*
* If there are inMemory segments, then they come first in the segments
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java
index ea8ef3afdcac..28ce8cd85eaf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java
@@ -40,10 +40,10 @@
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.mapred.SortedRanges.SkipRangeIterator;
@@ -56,6 +56,8 @@
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/** A Reduce task. */
@InterfaceAudience.Private
@InterfaceStability.Unstable
@@ -69,6 +71,7 @@ public class ReduceTask extends Task {
});
}
+ private static final XTrace.Logger xtrace = XTrace.getLogger(ReduceTask.class);
private static final Log LOG = LogFactory.getLog(ReduceTask.class.getName());
private int numMaps;
@@ -311,6 +314,9 @@ private void writeSkippedRec(KEY key, VALUE value) throws IOException{
@SuppressWarnings("unchecked")
public void run(JobConf job, final TaskUmbilicalProtocol umbilical)
throws IOException, InterruptedException, ClassNotFoundException {
+
+ xtrace.log("ReduceTask running");
+
job.setBoolean(JobContext.SKIP_RECORDS, isSkipping());
if (isMapOrReduce()) {
@@ -641,10 +647,12 @@ public boolean next() throws IOException {
committer,
reporter, comparator, keyClass,
valueClass);
+ xtrace.log("Running reduce start");
try {
reducer.run(reducerContext);
} finally {
trackedRW.close(reducerContext);
+ xtrace.log("Running reduce end");
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java
index 9e04e6fb87f7..4dea2b635ebf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java
@@ -106,7 +106,7 @@ public int size() {
public IndexRecord getIndex(int partition) {
final int pos = partition * MapTask.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8;
return new IndexRecord(entries.get(pos), entries.get(pos + 1),
- entries.get(pos + 2));
+ entries.get(pos + 2), entries.get(pos+3), entries.get(pos+4));
}
/**
@@ -117,6 +117,8 @@ public void putIndex(IndexRecord rec, int partition) {
entries.put(pos, rec.startOffset);
entries.put(pos + 1, rec.rawLength);
entries.put(pos + 2, rec.partLength);
+ entries.put(pos + 3, rec.getXTraceTaskID());
+ entries.put(pos + 4, rec.getXTraceOpID());
}
/**
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
index 685e61cfb63d..10f21def0ba1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
@@ -25,7 +25,9 @@
import java.lang.management.ManagementFactory;
import java.text.NumberFormat;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -41,9 +43,9 @@
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.RawComparator;
@@ -54,20 +56,23 @@
import org.apache.hadoop.io.serializer.SerializationFactory;
import org.apache.hadoop.mapred.IFile.Writer;
import org.apache.hadoop.mapreduce.FileSystemCounter;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.JobStatus;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
-import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
/**
* Base class for tasks.
@@ -75,8 +80,8 @@
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
abstract public class Task implements Writable, Configurable {
- private static final Log LOG =
- LogFactory.getLog(Task.class);
+ private static final XTrace.Logger xtrace = XTrace.getLogger(Task.class);
+ private static final Log LOG = LogFactory.getLog(Task.class);
public static String MERGED_OUTPUT_PREFIX = ".merged";
public static final long DEFAULT_COMBINE_RECORDS_BEFORE_PROGRESS = 10000;
@@ -139,6 +144,19 @@ static synchronized String getOutputName(int partition) {
return "part-" + NUMBER_FORMAT.format(partition);
}
+ ////////////////////////////////////////////
+ // X-Trace Methods
+ ////////////////////////////////////////////
+ private Collection xtrace_contexts = new HashSet();
+ public void rememberContext() {
+ xtrace_contexts.add(XTrace.get());
+ }
+ public void joinContext() {
+ for (Context ctx : xtrace_contexts)
+ XTrace.join(ctx);
+ }
+
+
////////////////////////////////////////////
// Fields
////////////////////////////////////////////
@@ -321,6 +339,7 @@ protected void setWriteSkipRecs(boolean writeSkipRecs) {
protected void reportFatalError(TaskAttemptID id, Throwable throwable,
String logMsg) {
LOG.fatal(logMsg);
+ xtrace.log(logMsg);
Throwable tCause = throwable.getCause();
String cause = tCause == null
? StringUtils.stringifyException(throwable)
@@ -329,6 +348,9 @@ protected void reportFatalError(TaskAttemptID id, Throwable throwable,
umbilical.fatalError(id, cause);
} catch (IOException ioe) {
LOG.fatal("Failed to contact the tasktracker", ioe);
+ xtrace.log("Failed to contact the tasktracker", "Exit Code", -1);
+ // TODO: XTrace parent process stuff
+// XTraceContext.joinParentProcess();
System.exit(-1);
}
}
@@ -740,7 +762,10 @@ public void run() {
// came back up), kill ourselves
if (!taskFound) {
LOG.warn("Parent died. Exiting "+taskId);
+ xtrace.log("Parent died, exiting", "Exit Code", 66);
resetDoneFlag();
+// // TODO: XTrace join parent process
+// XTraceContext.joinParentProcess();
System.exit(66);
}
@@ -750,10 +775,14 @@ public void run() {
catch (Throwable t) {
LOG.info("Communication exception: " + StringUtils.stringifyException(t));
remainingRetries -=1;
+ xtrace.log("Communication exception", "Throwable", t.getClass().getName(), "Message", t.getMessage(), "Retries Remaining", remainingRetries);
if (remainingRetries == 0) {
ReflectionUtils.logThreadInfo(LOG, "Communication exception", 0);
LOG.warn("Last retry, killing "+taskId);
+ xtrace.log("No retries remaining, killing task", "Exit Code", 65);
resetDoneFlag();
+// // TODO: XTrace join parent process
+// XTraceContext.joinParentProcess();
System.exit(65);
}
}
@@ -994,6 +1023,8 @@ public void done(TaskUmbilicalProtocol umbilical,
) throws IOException, InterruptedException {
LOG.info("Task:" + taskId + " is done."
+ " And is in the process of committing");
+
+ xtrace.log("Task is done and in the process of committing");
updateCounters();
boolean commitRequired = isCommitRequired();
@@ -1010,7 +1041,11 @@ public void done(TaskUmbilicalProtocol umbilical,
} catch (IOException ie) {
LOG.warn("Failure sending commit pending: " +
StringUtils.stringifyException(ie));
+ xtrace.log("Failure sending commit pending: "+ie.getClass().getName(), "Message", ie.getMessage(), "Retries Remaining", retries);
if (--retries == 0) {
+ xtrace.log("No retries remaining for task commit, killing task", "Exit Code", 67);
+// // TODO: Xtrace join parent process
+// XTraceContext.joinParentProcess();
System.exit(67);
}
}
@@ -1056,6 +1091,9 @@ public void statusUpdate(TaskUmbilicalProtocol umbilical)
try {
if (!umbilical.statusUpdate(getTaskID(), taskStatus)) {
LOG.warn("Parent died. Exiting "+taskId);
+ xtrace.log("Parent died, exiting", "Exit Code", 66);
+// // TODO: XTrace join parent process
+// XTraceContext.joinParentProcess();
System.exit(66);
}
taskStatus.clearStatus();
@@ -1111,12 +1149,14 @@ private void sendDone(TaskUmbilicalProtocol umbilical) throws IOException {
int retries = MAX_RETRIES;
while (true) {
try {
+ xtrace.log("Notifying AppMaster that task is done");
umbilical.done(getTaskID());
LOG.info("Task '" + taskId + "' done.");
return;
} catch (IOException ie) {
LOG.warn("Failure signalling completion: " +
StringUtils.stringifyException(ie));
+ xtrace.log("Failure signalling completion: "+ie.getClass().getName(), "Message", ie.getMessage(), "Retries Remaining", retries);
if (--retries == 0) {
throw ie;
}
@@ -1129,6 +1169,7 @@ private void commit(TaskUmbilicalProtocol umbilical,
org.apache.hadoop.mapreduce.OutputCommitter committer
) throws IOException {
int retries = MAX_RETRIES;
+ xtrace.log("Awaiting commit approval from AM");
while (true) {
try {
while (!umbilical.canCommit(taskId)) {
@@ -1143,9 +1184,14 @@ private void commit(TaskUmbilicalProtocol umbilical,
} catch (IOException ie) {
LOG.warn("Failure asking whether task can commit: " +
StringUtils.stringifyException(ie));
+ xtrace.log("Failure asking whether task can commit: "+ie.getClass().getName(),
+ "Message", ie.getMessage(), "Retries Remaining", retries);
if (--retries == 0) {
//if it couldn't query successfully then delete the output
discardOutput(taskContext);
+ xtrace.log("Maximum retries reached, discarding output and exiting", "Exit Code", 68);
+// // TODO: XTrace join parent process
+// XTraceContext.joinParentProcess();
System.exit(68);
}
}
@@ -1154,11 +1200,13 @@ private void commit(TaskUmbilicalProtocol umbilical,
// task can Commit now
try {
LOG.info("Task " + taskId + " is allowed to commit now");
+ xtrace.log("Task is allowed to commit, committing");
committer.commitTask(taskContext);
return;
} catch (IOException iee) {
LOG.warn("Failure committing: " +
StringUtils.stringifyException(iee));
+ xtrace.log("Failure committing: "+iee.getClass().getName()+", discarding output", "Message", iee.getMessage());
//if it couldn't commit a successfully then delete the output
discardOutput(taskContext);
throw iee;
@@ -1189,6 +1237,7 @@ void taskCleanup(TaskUmbilicalProtocol umbilical)
getProgress().setStatus("cleanup");
statusUpdate(umbilical);
LOG.info("Runnning cleanup for the task");
+ xtrace.log("Runnning cleanup for the task");
// do the cleanup
committer.abortTask(taskContext);
}
@@ -1202,9 +1251,11 @@ protected void runJobCleanupTask(TaskUmbilicalProtocol umbilical,
statusUpdate(umbilical);
// do the cleanup
LOG.info("Cleaning up job");
+ xtrace.log("Cleaning up job");
if (jobRunStateForCleanup == JobStatus.State.FAILED
|| jobRunStateForCleanup == JobStatus.State.KILLED) {
LOG.info("Aborting job with runstate : " + jobRunStateForCleanup.name());
+ xtrace.log("Aborting job with runstate : " + jobRunStateForCleanup.name());
if (conf.getUseNewMapper()) {
committer.abortJob(jobContext, jobRunStateForCleanup);
} else {
@@ -1213,9 +1264,15 @@ protected void runJobCleanupTask(TaskUmbilicalProtocol umbilical,
oldCommitter.abortJob(jobContext, jobRunStateForCleanup);
}
} else if (jobRunStateForCleanup == JobStatus.State.SUCCEEDED){
+ xtrace.log("Committing job");
LOG.info("Committing job");
committer.commitJob(jobContext);
} else {
+ xtrace.log("Invalid state of the job for cleanup. Found state "
+ + jobRunStateForCleanup + ", but was expecting "
+ + JobStatus.State.SUCCEEDED + ", "
+ + JobStatus.State.FAILED + " or "
+ + JobStatus.State.KILLED);
throw new IOException("Invalid state of the job for cleanup. State found "
+ jobRunStateForCleanup + " expecting "
+ JobStatus.State.SUCCEEDED + ", "
@@ -1641,7 +1698,10 @@ public void combine(RawKeyValueIterator iterator,
committer,
reporter, comparator, keyClass,
valueClass);
+ xtrace.log("Combining map outputs",
+ "Combiner", reducer.getClass().getName(), "KeyClass", keyClass, "ValClass", valueClass);
reducer.run(reducerContext);
+ xtrace.log("Combine complete");
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultithreadedMapRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultithreadedMapRunner.java
index 456cdfab9a87..9c241bc811b5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultithreadedMapRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultithreadedMapRunner.java
@@ -18,22 +18,25 @@
package org.apache.hadoop.mapred.lib;
-import org.apache.hadoop.util.ReflectionUtils;
+import java.io.IOException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.mapred.MapRunnable;
import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapRunnable;
import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.SkipBadRecords;
import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.io.IOException;
-import java.util.concurrent.*;
+import org.apache.hadoop.util.ReflectionUtils;
/**
* Multithreaded implementation for @link org.apache.hadoop.mapred.MapRunnable.
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
index e93f27360442..0a0e9dab8e02 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
@@ -75,10 +75,14 @@ public Cluster(Configuration conf) throws IOException {
}
public Cluster(InetSocketAddress jobTrackAddr, Configuration conf)
- throws IOException {
+ throws IOException {
this.conf = conf;
this.ugi = UserGroupInformation.getCurrentUser();
- initialize(jobTrackAddr, conf);
+ try {
+ initialize(jobTrackAddr, conf);
+ } catch (IOException e) {
+ throw e;
+ }
}
private void initialize(InetSocketAddress jobTrackAddr, Configuration conf)
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
index 78c6b4b1a9c5..e1e85fc011df 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
@@ -19,14 +19,15 @@
package org.apache.hadoop.mapreduce;
import java.io.IOException;
+import java.io.StringWriter;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configuration.IntegerRanges;
import org.apache.hadoop.fs.FileSystem;
@@ -39,6 +40,9 @@
import org.apache.hadoop.mapreduce.util.ConfigUtil;
import org.apache.hadoop.util.StringUtils;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* The job submitter's view of the Job.
*
@@ -75,6 +79,7 @@
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class Job extends JobContextImpl implements JobContext {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(Job.class);
private static final Log LOG = LogFactory.getLog(Job.class);
@InterfaceStability.Evolving
@@ -110,6 +115,7 @@ public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL }
private JobState state = JobState.DEFINE;
private JobStatus status;
+ private Context status_xtrace;
private long statustime;
private Cluster cluster;
@@ -134,6 +140,15 @@ public Job(Configuration conf, String jobName) throws IOException {
// propagate existing user credentials to job
this.credentials.mergeAll(this.ugi.getCredentials());
this.cluster = null;
+ StringWriter partial = new StringWriter();
+ conf.writeXml(partial);
+ StringWriter full = new StringWriter();
+ Configuration.dumpConfiguration(conf, full);
+ if (!"".equals(conf.getJobName())) {
+ xtrace.log("Initializing Job", conf.getJobName());
+ } else {
+ xtrace.log("Initializing Job");
+ }
}
Job(JobStatus status, JobConf conf) throws IOException {
@@ -307,6 +322,8 @@ synchronized void ensureFreshStatus()
* @throws IOException
*/
synchronized void updateStatus() throws IOException {
+ Context start_context = XTrace.get();
+ XTrace.stop();
try {
this.status = ugi.doAs(new PrivilegedExceptionAction() {
@Override
@@ -317,6 +334,9 @@ public JobStatus run() throws IOException, InterruptedException {
}
catch (InterruptedException ie) {
throw new IOException(ie);
+ } finally {
+ this.status_xtrace = XTrace.get();
+ XTrace.set(start_context);
}
if (this.status == null) {
throw new IOException("Job status not available ");
@@ -330,6 +350,10 @@ public JobStatus getStatus() throws IOException, InterruptedException {
return status;
}
+ public void joinStatusXTraceContext() {
+ XTrace.join(this.status_xtrace);
+ }
+
private void setStatus(JobStatus status) {
this.status = status;
}
@@ -1282,9 +1306,12 @@ public JobStatus run() throws IOException, InterruptedException,
public boolean waitForCompletion(boolean verbose
) throws IOException, InterruptedException,
ClassNotFoundException {
+
+ xtrace.log("Preparing Job");
if (state == JobState.DEFINE) {
submit();
}
+ xtrace.log("Submitted Job", "Job ID", getJobID());
if (verbose) {
monitorAndPrintJob();
} else {
@@ -1298,6 +1325,15 @@ public boolean waitForCompletion(boolean verbose
}
}
}
+
+ long maps = getCounters().findCounter(JobCounter.TOTAL_LAUNCHED_MAPS).getValue();
+ long reds = getCounters().findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES).getValue();
+ if (isSuccessful()) {
+ xtrace.log("Job finished successfully", "Tag", maps+" Maps", "Tag", reds+" Reduces");
+ } else {
+ xtrace.log("Job failed", "Tag", maps+" Maps", "Tag", reds+" Reduces");
+
+ }
return isSuccessful();
}
@@ -1351,12 +1387,15 @@ public boolean monitorAndPrintJob()
eventCounter += events.length;
printTaskEvents(events, filter, profiling, mapRanges, reduceRanges);
}
+ joinStatusXTraceContext();
boolean success = isSuccessful();
if (success) {
LOG.info("Job " + jobId + " completed successfully");
+ xtrace.log("Job completed successfully");
} else {
LOG.info("Job " + jobId + " failed with state " + status.getState() +
" due to: " + status.getFailureInfo());
+ xtrace.log("Job failed with state "+status.getState());
}
Counters counters = getCounters();
if (counters != null) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
index 7e0453d3bfb8..04144a66c0a3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
@@ -20,17 +20,15 @@
import java.io.IOException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
/**
* A utility to manage job submission files.
*/
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
index cee7a33d7e6c..3abd5c853252 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.mapreduce;
+import static org.apache.hadoop.mapred.QueueManager.toFullPropertyName;
+
import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
@@ -45,8 +47,6 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.QueueACL;
-import static org.apache.hadoop.mapred.QueueManager.toFullPropertyName;
-
import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager;
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
@@ -63,9 +63,13 @@
import com.google.common.base.Charsets;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
@InterfaceAudience.Private
@InterfaceStability.Unstable
class JobSubmitter {
+ protected static final XTrace.Logger xtrace = XTrace.getLogger(JobSubmitter.class);
protected static final Log LOG = LogFactory.getLog(JobSubmitter.class);
private static final String SHUFFLE_KEYGEN_ALGORITHM = "HmacSHA1";
private static final int SHUFFLE_KEY_LENGTH = 64;
@@ -336,7 +340,10 @@ private void copyAndConfigureFiles(Job job, Path jobSubmitDir)
JobStatus submitJobInternal(Job job, Cluster cluster)
throws ClassNotFoundException, InterruptedException, IOException {
- //validate the jobs output specs
+ xtrace.log("Submitting Job");
+ Context start_context = XTrace.get();
+
+ //validate the jobs output specs
checkSpecs(job);
Path jobStagingArea = JobSubmissionFiles.getStagingDir(cluster,
@@ -351,6 +358,7 @@ JobStatus submitJobInternal(Job job, Cluster cluster)
conf.set(MRJobConfig.JOB_SUBMITHOSTADDR,submitHostAddress);
}
JobID jobId = submitClient.getNewJobID();
+ System.out.println("Submitting job with jobid " + jobId + " and the client class is " + submitClient.getClass().getName());
job.setJobID(jobId);
Path submitJobDir = new Path(jobStagingArea, jobId.toString());
JobStatus status = null;
@@ -426,6 +434,9 @@ JobStatus submitJobInternal(Job job, Cluster cluster)
jtFs.delete(submitJobDir, true);
}
+
+ XTrace.join(start_context);
+ xtrace.log("Job submission complete");
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java
index 3a6186b9b915..476b65fc081c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java
@@ -27,6 +27,8 @@
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.mapreduce.task.MapContextImpl;
+import edu.brown.cs.systems.resourcetracing.CPUTracking;
+
/**
* Maps input key/value pairs to a set of intermediate key/value pairs.
*
@@ -142,6 +144,7 @@ public void run(Context context) throws IOException, InterruptedException {
setup(context);
try {
while (context.nextKeyValue()) {
+ CPUTracking.continueTracking();
map(context.getCurrentKey(), context.getCurrentValue(), context);
}
} finally {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java
index ddf67e18abb2..8a01065db2c3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java
@@ -25,6 +25,8 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.task.annotation.Checkpointable;
+import edu.brown.cs.systems.resourcetracing.CPUTracking;
+
import java.util.Iterator;
/**
@@ -168,6 +170,7 @@ public void run(Context context) throws IOException, InterruptedException {
setup(context);
try {
while (context.nextKey()) {
+ CPUTracking.continueTracking();
reduce(context.getCurrentKey(), context.getValues(), context);
// If a back up store is used, reset it
Iterator iter = context.getValues().iterator();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java
index 814e49451230..97897976679c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java
@@ -18,7 +18,12 @@
package org.apache.hadoop.mapreduce.lib.map;
-import org.apache.hadoop.util.ReflectionUtils;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@@ -33,12 +38,7 @@
import org.apache.hadoop.mapreduce.StatusReporter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
+import org.apache.hadoop.util.ReflectionUtils;
/**
* Multithreaded implementation for @link org.apache.hadoop.mapreduce.Mapper.
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
index 4f1e5d14efad..03f99d6c299d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
@@ -26,6 +26,7 @@
import java.net.URLConnection;
import java.security.GeneralSecurityException;
import java.util.Arrays;
+import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@@ -47,8 +48,12 @@
import com.google.common.annotations.VisibleForTesting;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
class Fetcher extends Thread {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(Fetcher.class);
private static final Log LOG = LogFactory.getLog(Fetcher.class);
/** Number of ms before timing out a copy */
@@ -87,6 +92,9 @@ private static enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
protected HttpURLConnection connection;
private volatile boolean stopped = false;
+ private final Context xtrace_start_context;
+ private final Collection copy_contexts = new HashSet();
+
private static boolean sslShuffle;
private static SSLFactory sslFactory;
@@ -147,6 +155,8 @@ public Fetcher(JobConf job, TaskAttemptID reduceId,
}
}
}
+
+ xtrace_start_context = XTrace.get();
}
public void run() {
@@ -156,6 +166,8 @@ public void run() {
try {
// If merge is on, block
merger.waitForResource();
+
+ XTrace.set(xtrace_start_context);
// Get a host to shuffle from
host = scheduler.getHost();
@@ -168,6 +180,8 @@ public void run() {
scheduler.freeHost(host);
metrics.threadFree();
}
+ copy_contexts.add(XTrace.get());
+ XTrace.stop();
}
}
} catch (InterruptedException ie) {
@@ -198,6 +212,11 @@ public void shutDown() throws InterruptedException {
sslFactory.destroy();
}
}
+
+ public void joinContexts() {
+ for (Context ctx : copy_contexts)
+ XTrace.join(ctx);
+ }
@VisibleForTesting
protected synchronized void openConnection(URL url)
@@ -251,6 +270,7 @@ protected void copyFromHost(MapHost host) throws IOException {
LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: "
+ maps);
}
+ xtrace.log("Fetching map outputs from mapper", "Num Maps", maps.size(), "Host", host, "TaskAttemptIDs", maps);
// List of maps to be fetched yet
Set remaining = new HashSet(maps);
@@ -259,6 +279,7 @@ protected void copyFromHost(MapHost host) throws IOException {
DataInputStream input = null;
try {
URL url = getMapOutputURL(host, maps);
+ xtrace.log("Connecting to map output on host", "URL", url);
openConnection(url);
if (stopped) {
abortConnect(host, remaining);
@@ -273,6 +294,9 @@ protected void copyFromHost(MapHost host) throws IOException {
// put url hash into http header
connection.addRequestProperty(
SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash);
+ String xtrace_string = XTrace.base16();
+ if (xtrace_string!=null)
+ connection.addRequestProperty("X-Trace", xtrace_string);
// set the read timeout
connection.setReadTimeout(readTimeout);
// put shuffle version into http header
@@ -291,6 +315,11 @@ protected void copyFromHost(MapHost host) throws IOException {
// Validate response code
int rc = connection.getResponseCode();
if (rc != HttpURLConnection.HTTP_OK) {
+ String xtrace_context = connection.getHeaderField("X-Trace");
+ if (xtrace_context!=null) {
+ XTrace.join(Context.parseBase16(xtrace_context));
+ }
+ xtrace.log("Got invalid response code " + rc + " from host", "URL", url, "Message", connection.getResponseMessage());
throw new IOException(
"Got invalid response code " + rc + " from " + url +
": " + connection.getResponseMessage());
@@ -316,6 +345,7 @@ protected void copyFromHost(MapHost host) throws IOException {
ioErrs.increment(1);
LOG.warn("Failed to connect to " + host + " with " + remaining.size() +
" map outputs", ie);
+ xtrace.log("Failed to connect to host: "+ie.getClass().getName(), "Host", host, "Remaining Outputs", remaining.size(), "Message", ie.getMessage());
// If connect did not succeed, just mark all the maps as failed,
// indirectly penalizing the host
@@ -337,12 +367,20 @@ protected void copyFromHost(MapHost host) throws IOException {
// after putting back the remaining maps to the
// yet_to_be_fetched list and marking the failed tasks.
TaskAttemptID[] failedTasks = null;
+ Context start_context = XTrace.get();
+ Collection end_contexts = new HashSet();
+ int initialSize = remaining.size();
while (!remaining.isEmpty() && failedTasks == null) {
+ XTrace.set(start_context);
failedTasks = copyMapOutput(host, input, remaining);
+ end_contexts.add(XTrace.get());
}
+ for (Context ctx : end_contexts)
+ XTrace.join(ctx);
if(failedTasks != null && failedTasks.length > 0) {
LOG.warn("copyMapOutput failed for tasks "+Arrays.toString(failedTasks));
+ xtrace.log("Failed to copy map output for some tasks", "Failed Tasks", Arrays.toString(failedTasks));
for(TaskAttemptID left: failedTasks) {
scheduler.copyFailed(left, host, true, false);
}
@@ -350,9 +388,16 @@ protected void copyFromHost(MapHost host) throws IOException {
// Sanity check
if (failedTasks == null && !remaining.isEmpty()) {
+ xtrace.log("Server didn't return all expected map outputs", "Remaining", remaining.size());
throw new IOException("server didn't return all expected map outputs: "
+ remaining.size() + " left.");
}
+
+
+ int failed = remaining.size();
+ int copied = initialSize - failed;
+ xtrace.log("Fetching complete", "Num Succeeded", copied, "Num Failed", failed);
+
input.close();
input = null;
} finally {
@@ -390,6 +435,7 @@ private TaskAttemptID[] copyMapOutput(MapHost host,
} catch (IllegalArgumentException e) {
badIdErrs.increment(1);
LOG.warn("Invalid map id ", e);
+ xtrace.log("Invalid map ID: "+e.getClass().getName(), "Message", e.toString(), "Map ID", mapId);
//Don't know which one was bad, so consider all of them as bad
return remaining.toArray(new TaskAttemptID[remaining.size()]);
}
@@ -407,11 +453,12 @@ private TaskAttemptID[] copyMapOutput(MapHost host,
}
// Get the location for the map output - either in-memory or on-disk
+ xtrace.log("Reserving location for map output");
mapOutput = merger.reserve(mapId, decompressedLength, id);
// Check if we can shuffle *now* ...
if (mapOutput == null) {
- LOG.info("fetcher#" + id + " - MergeManager returned status WAIT ...");
+ xtrace.log("Merge Manager instructed fetcher to wait", "Fetcher ID", id);
//Not an error but wait to process data.
return EMPTY_ATTEMPT_ID_ARRAY;
}
@@ -424,6 +471,9 @@ private TaskAttemptID[] copyMapOutput(MapHost host,
LOG.info("fetcher#" + id + " about to shuffle output of map "
+ mapOutput.getMapId() + " decomp: " + decompressedLength
+ " len: " + compressedLength + " to " + mapOutput.getDescription());
+ xtrace.log("Shuffling ouputs from mapper",
+ "Fetcher ID", id, "Map ID", mapOutput.getMapId(), "Decompressed Length", decompressedLength,
+ "Compressed Length", compressedLength, "Copy Destination", mapOutput.getDescription());
mapOutput.shuffle(host, input, compressedLength, decompressedLength,
metrics, reporter);
} catch (java.lang.InternalError e) {
@@ -445,6 +495,9 @@ private TaskAttemptID[] copyMapOutput(MapHost host,
LOG.info("fetcher#" + id + " failed to read map header" +
mapId + " decomp: " +
decompressedLength + ", " + compressedLength, ioe);
+ xtrace.log("Fetcher failed to read map header: "+ioe.getClass().getName(),
+ "Fetcher ID", id, "Map ID", mapId, "Decompressed Length", decompressedLength, "Compressed Length", compressedLength,
+ "Message", ioe.getMessage());
if(mapId == null) {
return remaining.toArray(new TaskAttemptID[remaining.size()]);
} else {
@@ -454,6 +507,8 @@ private TaskAttemptID[] copyMapOutput(MapHost host,
LOG.warn("Failed to shuffle output of " + mapId +
" from " + host.getHostName(), ioe);
+ xtrace.log("Failed failed to shuffle map output: "+ioe.getClass().getName(),
+ "Fetcher Id", id, "Map ID", mapId, "Host", host.getHostName(), "Message", ioe.getMessage());
// Inform the shuffle-scheduler
mapOutput.abort();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/InMemoryMapOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/InMemoryMapOutput.java
index 24fb3bbaca93..9fdfd5e6a52c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/InMemoryMapOutput.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/InMemoryMapOutput.java
@@ -17,32 +17,29 @@
*/
package org.apache.hadoop.mapreduce.task.reduce;
-import java.io.InputStream;
import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import java.io.InputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
-
import org.apache.hadoop.io.BoundedByteArrayOutputStream;
import org.apache.hadoop.io.IOUtils;
-
import org.apache.hadoop.io.compress.CodecPool;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.Decompressor;
-
import org.apache.hadoop.mapred.IFileInputStream;
import org.apache.hadoop.mapred.Reporter;
-
import org.apache.hadoop.mapreduce.TaskAttemptID;
+import edu.brown.cs.systems.xtrace.XTrace;
+
@InterfaceAudience.Private
@InterfaceStability.Unstable
class InMemoryMapOutput extends MapOutput {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(InMemoryMapOutput.class);
private static final Log LOG = LogFactory.getLog(InMemoryMapOutput.class);
private Configuration conf;
private final MergeManagerImpl merger;
@@ -125,6 +122,7 @@ public void shuffle(MapHost host, InputStream input,
@Override
public void commit() throws IOException {
+ xtrace.log("Map output committed");
merger.closeInMemoryFile(this);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MapOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MapOutput.java
index b5a8cf539999..89b10c3b75a6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MapOutput.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MapOutput.java
@@ -17,19 +17,19 @@
*/
package org.apache.hadoop.mapreduce.task.reduce;
-import java.io.InputStream;
import java.io.IOException;
-
+import java.io.InputStream;
import java.util.Comparator;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-
import org.apache.hadoop.mapred.Reporter;
-
import org.apache.hadoop.mapreduce.TaskAttemptID;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
public abstract class MapOutput {
@@ -40,6 +40,8 @@ public abstract class MapOutput {
private final long size;
private final boolean primaryMapOutput;
+ private Context xtrace_context = null;
+
public MapOutput(TaskAttemptID mapId, long size, boolean primaryMapOutput) {
this.id = ID.incrementAndGet();
this.mapId = mapId;
@@ -47,6 +49,14 @@ public MapOutput(TaskAttemptID mapId, long size, boolean primaryMapOutput) {
this.primaryMapOutput = primaryMapOutput;
}
+ public void rememberContext() {
+ xtrace_context = XTrace.get();
+ }
+
+ public void joinContext() {
+ XTrace.join(xtrace_context);
+ }
+
public boolean isPrimaryMapOutput() {
return primaryMapOutput;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
index e52f4bf6593d..44e0c3508e1b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
@@ -39,16 +39,16 @@
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.mapred.Counters;
import org.apache.hadoop.mapred.IFile;
+import org.apache.hadoop.mapred.IFile.Reader;
+import org.apache.hadoop.mapred.IFile.Writer;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.MapOutputFile;
import org.apache.hadoop.mapred.Merger;
+import org.apache.hadoop.mapred.Merger.Segment;
import org.apache.hadoop.mapred.RawKeyValueIterator;
import org.apache.hadoop.mapred.Reducer;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.Task;
-import org.apache.hadoop.mapred.IFile.Reader;
-import org.apache.hadoop.mapred.IFile.Writer;
-import org.apache.hadoop.mapred.Merger.Segment;
import org.apache.hadoop.mapred.Task.CombineOutputCollector;
import org.apache.hadoop.mapred.Task.CombineValuesIterator;
import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -60,11 +60,15 @@
import com.google.common.annotations.VisibleForTesting;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
@SuppressWarnings(value={"unchecked"})
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
public class MergeManagerImpl implements MergeManager {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(MergeManagerImpl.class);
private static final Log LOG = LogFactory.getLog(MergeManagerImpl.class);
/* Maximum percentage of the in-memory limit that a single shuffle can
@@ -254,6 +258,8 @@ public synchronized MapOutput reserve(TaskAttemptID mapId,
LOG.info(mapId + ": Shuffling to disk since " + requestedSize +
" is greater than maxSingleShuffleLimit (" +
maxSingleShuffleLimit + ")");
+ xtrace.log("Shuffling directly to disk due to reservation size exceeding max single shuffle limit",
+ "Map ID", mapId, "Requested Size", requestedSize, "maxSingleShuffleLimit", maxSingleShuffleLimit);
return new OnDiskMapOutput(mapId, reduceId, this, requestedSize,
jobConf, mapOutputFile, fetcher, true);
}
@@ -276,7 +282,9 @@ public synchronized MapOutput reserve(TaskAttemptID mapId,
if (usedMemory > memoryLimit) {
LOG.debug(mapId + ": Stalling shuffle since usedMemory (" + usedMemory
+ ") is greater than memoryLimit (" + memoryLimit + ")." +
- " CommitMemory is (" + commitMemory + ")");
+ " CommitMemory is (" + commitMemory + ")");
+ xtrace.log("Currently above memory limit, stalling shuffle", "Map ID", mapId,
+ "Used Memory", usedMemory, "Memory Limit", memoryLimit, "Commit Memory", commitMemory);
return null;
}
@@ -284,6 +292,8 @@ public synchronized MapOutput reserve(TaskAttemptID mapId,
LOG.debug(mapId + ": Proceeding with shuffle since usedMemory ("
+ usedMemory + ") is lesser than memoryLimit (" + memoryLimit + ")."
+ "CommitMemory is (" + commitMemory + ")");
+ xtrace.log("Proceeding with shuffle", "Map ID", mapId, "Used Memory", usedMemory,
+ "Memory Limit", memoryLimit, "Commit Memory", commitMemory);
return unconditionalReserve(mapId, requestedSize, true);
}
@@ -302,7 +312,8 @@ synchronized void unreserve(long size) {
usedMemory -= size;
}
- public synchronized void closeInMemoryFile(InMemoryMapOutput mapOutput) {
+ public synchronized void closeInMemoryFile(InMemoryMapOutput mapOutput) {
+ mapOutput.rememberContext();
inMemoryMapOutputs.add(mapOutput);
LOG.info("closeInMemoryFile -> map-output of size: " + mapOutput.getSize()
+ ", inMemoryMapOutputs.size() -> " + inMemoryMapOutputs.size()
@@ -330,6 +341,7 @@ public synchronized void closeInMemoryFile(InMemoryMapOutput mapOutput) {
public synchronized void closeInMemoryMergedFile(InMemoryMapOutput mapOutput) {
+ mapOutput.rememberContext();
inMemoryMergedMapOutputs.add(mapOutput);
LOG.info("closeInMemoryMergedFile -> size: " + mapOutput.getSize() +
", inMemoryMergedMapOutputs.size() -> " +
@@ -337,6 +349,7 @@ public synchronized void closeInMemoryMergedFile(InMemoryMapOutput mapOutpu
}
public synchronized void closeOnDiskFile(CompressAwarePath file) {
+ file.rememberContext();
onDiskMapOutputs.add(file);
if (onDiskMapOutputs.size() >= (2 * ioSortFactor - 1)) {
@@ -391,6 +404,8 @@ public void merge(List> inputs) throws IOException {
LOG.info("Initiating Memory-to-Memory merge with " + noInMemorySegments +
" segments of total-size: " + mergeOutputSize);
+ xtrace.log("Initiating Memory-to-Memory merge", "Num Segments", noInMemorySegments, "Total Size", mergeOutputSize,
+ "Reduce ID", reduceId);
RawKeyValueIterator rIter =
Merger.merge(jobConf, rfs,
@@ -406,6 +421,7 @@ public void merge(List> inputs) throws IOException {
LOG.info(reduceId +
" Memory-to-Memory merge of the " + noInMemorySegments +
" files in-memory complete.");
+ xtrace.log("Memory-to-Memory merge complete", "Reduce ID", reduceId, "Num Segments", noInMemorySegments);
// Note the output of the merge
closeInMemoryMergedFile(mergedMapOutputs);
@@ -457,9 +473,11 @@ public void merge(List> inputs) throws IOException {
RawKeyValueIterator rIter = null;
CompressAwarePath compressAwarePath;
+
try {
LOG.info("Initiating in-memory merge with " + noInMemorySegments +
" segments...");
+ xtrace.log("Initiating in-memory merge", "Num Segments", noInMemorySegments);
rIter = Merger.merge(jobConf, rfs,
(Class)jobConf.getMapOutputKeyClass(),
@@ -484,12 +502,16 @@ public void merge(List> inputs) throws IOException {
" files in-memory complete." +
" Local file is " + outputPath + " of size " +
localFS.getFileStatus(outputPath).getLen());
+ xtrace.log("Merge complete", "Num Segments", noInMemorySegments,
+ "Local File", outputPath, "Size", localFS.getFileStatus(outputPath).getLen());
} catch (IOException e) {
//make sure that we delete the ondisk file that we created
//earlier when we invoked cloneFileAttributes
localFS.delete(outputPath, true);
throw e;
}
+
+ xtrace.log("In-memory merge finished");
// Note the output of the merge
closeOnDiskFile(compressAwarePath);
@@ -512,13 +534,19 @@ public void merge(List inputs) throws IOException {
LOG.info("No ondisk files to merge...");
return;
}
+ for (CompressAwarePath input : inputs) {
+ input.joinContext();
+ }
long approxOutputSize = 0;
int bytesPerSum =
jobConf.getInt("io.bytes.per.checksum", 512);
+
+
LOG.info("OnDiskMerger: We have " + inputs.size() +
" map outputs on disk. Triggering merge...");
+ xtrace.log("Merging outputs on disk", "Num Segments", inputs.size());
// 1. Prepare the list of files to be merged.
for (CompressAwarePath file : inputs) {
@@ -559,6 +587,9 @@ public void merge(List inputs) throws IOException {
localFS.delete(outputPath, true);
throw e;
}
+
+ xtrace.log("Finished merging map output files on disk",
+ "Num Segments", inputs.size(), "Output Size", approxOutputSize, "Output File", outputPath);
closeOnDiskFile(compressAwarePath);
@@ -607,6 +638,7 @@ private long createInMemorySegments(List> inMemoryMapOutp
}
while(fullSize > leaveBytes) {
InMemoryMapOutput mo = inMemoryMapOutputs.remove(0);
+ mo.joinContext();
byte[] data = mo.getMemory();
long size = data.length;
totalSize += size;
@@ -664,6 +696,8 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
LOG.info("finalMerge called with " +
inMemoryMapOutputs.size() + " in-memory map-outputs and " +
onDiskMapOutputs.size() + " on-disk map-outputs");
+ xtrace.log("Starting final merge",
+ "Num In-Memory Map Outputs", inMemoryMapOutputs.size(), "Num On-Disk Map Outputs", onDiskMapOutputs.size());
final float maxRedPer =
job.getFloat(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, 0f);
@@ -687,6 +721,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
List> memDiskSegments = new ArrayList>();
long inMemToDiskBytes = 0;
boolean mergePhaseFinished = false;
+ Context start_context = XTrace.get();
if (inMemoryMapOutputs.size() > 0) {
TaskID mapId = inMemoryMapOutputs.get(0).getMapId().getTaskID();
inMemToDiskBytes = createInMemorySegments(inMemoryMapOutputs,
@@ -695,7 +730,8 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
final int numMemDiskSegments = memDiskSegments.size();
if (numMemDiskSegments > 0 &&
ioSortFactor > onDiskMapOutputs.size()) {
-
+ xtrace.log("Merging segments to disk to satisfy reduce memory limit", "Num Segments", numMemDiskSegments,
+ "NumBytes", inMemToDiskBytes);
// If we reach here, it implies that we have less than io.sort.factor
// disk segments and this will be incremented by 1 (result of the
// memory segments merge). Since this total would still be
@@ -717,9 +753,12 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
keyClass, valueClass, codec, null);
try {
Merger.writeFile(rIter, writer, reporter, job);
+ xtrace.log("Merge complete");
writer.close();
- onDiskMapOutputs.add(new CompressAwarePath(outputPath,
- writer.getRawLength(), writer.getCompressedLength()));
+ CompressAwarePath p = new CompressAwarePath(outputPath, writer.getRawLength(), writer.getCompressedLength());
+ onDiskMapOutputs.add(p);
+ xtrace.log("Merged to disk", "OutputPath", outputPath.toString());
+ p.rememberContext();
writer = null;
// add to list of final disk outputs.
} catch (IOException e) {
@@ -741,6 +780,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
"reduce memory limit");
inMemToDiskBytes = 0;
memDiskSegments.clear();
+ XTrace.set(start_context);
} else if (inMemToDiskBytes != 0) {
LOG.info("Keeping " + numMemDiskSegments + " segments, " +
inMemToDiskBytes + " bytes in memory for " +
@@ -748,6 +788,8 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
}
}
+ start_context = XTrace.get();
+
// segments on disk
List> diskSegments = new ArrayList>();
long onDiskBytes = inMemToDiskBytes;
@@ -755,6 +797,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
CompressAwarePath[] onDisk = onDiskMapOutputs.toArray(
new CompressAwarePath[onDiskMapOutputs.size()]);
for (CompressAwarePath file : onDisk) {
+ file.joinContext();
long fileLength = fs.getFileStatus(file).getLen();
onDiskBytes += fileLength;
rawBytes += (file.getRawDataLength() > 0) ? file.getRawDataLength() : fileLength;
@@ -768,6 +811,8 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
}
LOG.info("Merging " + onDisk.length + " files, " +
onDiskBytes + " bytes from disk");
+ xtrace.log("Merging files from disk",
+ "Num Segments", onDisk.length, "Num Bytes", onDiskBytes);
Collections.sort(diskSegments, new Comparator>() {
public int compare(Segment o1, Segment o2) {
if (o1.getLength() == o2.getLength()) {
@@ -776,6 +821,9 @@ public int compare(Segment o1, Segment o2) {
return o1.getLength() < o2.getLength() ? -1 : 1;
}
});
+
+ Context disk_context = XTrace.get();
+ XTrace.set(start_context);
// build final list of segments from merged backed by disk + in-mem
List> finalSegments = new ArrayList>();
@@ -783,7 +831,11 @@ public int compare(Segment o1, Segment o2) {
finalSegments, 0);
LOG.info("Merging " + finalSegments.size() + " segments, " +
inMemBytes + " bytes from memory into reduce");
+
+ Context mem_context = XTrace.get();
+
if (0 != onDiskBytes) {
+ XTrace.set(disk_context);
final int numInMemSegments = memDiskSegments.size();
diskSegments.addAll(0, memDiskSegments);
memDiskSegments.clear();
@@ -795,28 +847,42 @@ public int compare(Segment o1, Segment o2) {
ioSortFactor, numInMemSegments, tmpDir, comparator,
reporter, false, spilledRecordsCounter, null, thisPhase);
diskSegments.clear();
+ XTrace.join(mem_context);
+ xtrace.log("Piping segment merge into reduce", "Num Segments", finalSegments.size(), "Num Bytes", inMemBytes);
if (0 == finalSegments.size()) {
+ xtrace.log("Final merge complete, returning result iterator");
return diskMerge;
}
finalSegments.add(new Segment(
new RawKVIteratorReader(diskMerge, onDiskBytes), true, rawBytes));
}
- return Merger.merge(job, fs, keyClass, valueClass,
+ RawKeyValueIterator result = Merger.merge(job, fs, keyClass, valueClass,
finalSegments, finalSegments.size(), tmpDir,
comparator, reporter, spilledRecordsCounter, null,
null);
-
+
+ xtrace.log("Final merge complete, returning result iterator");
+ return result;
}
static class CompressAwarePath extends Path {
private long rawDataLength;
private long compressedSize;
+ private Context xtrace_context;
public CompressAwarePath(Path path, long rawDataLength, long compressSize) {
super(path.toUri());
this.rawDataLength = rawDataLength;
this.compressedSize = compressSize;
}
+
+ public void rememberContext() {
+ xtrace_context = XTrace.get();
+ }
+
+ public void joinContext() {
+ XTrace.join(xtrace_context);
+ }
public long getRawDataLength() {
return rawDataLength;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeThread.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeThread.java
index 5db353f99c21..419c35f1dd94 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeThread.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeThread.java
@@ -28,8 +28,11 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import edu.brown.cs.systems.xtrace.XTrace;
+
abstract class MergeThread extends Thread {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(MergeThread.class);
private static final Log LOG = LogFactory.getLog(MergeThread.class);
private AtomicInteger numPending = new AtomicInteger(0);
@@ -72,8 +75,11 @@ public void startMerge(Set inputs) {
}
public synchronized void waitForMerge() throws InterruptedException {
- while (numPending.get() > 0) {
+ int numPending = this.numPending.get();
+ while (numPending > 0) {
+ xtrace.log("Waiting for pending merges", "Num Pending", numPending);
wait();
+ numPending = this.numPending.get();
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/OnDiskMapOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/OnDiskMapOutput.java
index 59bb04a9dea4..ca60016b411b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/OnDiskMapOutput.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/OnDiskMapOutput.java
@@ -21,29 +21,27 @@
import java.io.InputStream;
import java.io.OutputStream;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-
import org.apache.hadoop.io.IOUtils;
-
import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.MapOutputFile;
-
+import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.task.reduce.MergeManagerImpl.CompressAwarePath;
import com.google.common.annotations.VisibleForTesting;
+import edu.brown.cs.systems.xtrace.XTrace;
+
@InterfaceAudience.Private
@InterfaceStability.Unstable
class OnDiskMapOutput extends MapOutput {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(OnDiskMapOutput.class);
private static final Log LOG = LogFactory.getLog(OnDiskMapOutput.class);
private final FileSystem fs;
private final Path tmpOutputPath;
@@ -134,6 +132,7 @@ public void commit() throws IOException {
CompressAwarePath compressAwarePath = new CompressAwarePath(outputPath,
getSize(), this.compressedSize);
merger.closeOnDiskFile(compressAwarePath);
+ xtrace.log("Map output committed", "tmpOutputPath", tmpOutputPath, "outputPath", outputPath);
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java
index 06c007e1584e..a1c1f57b4cc6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java
@@ -24,18 +24,21 @@
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RawKeyValueIterator;
import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.ShuffleConsumerPlugin;
import org.apache.hadoop.mapred.Task;
import org.apache.hadoop.mapred.TaskStatus;
import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
-import org.apache.hadoop.mapred.ShuffleConsumerPlugin;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.util.Progress;
+import edu.brown.cs.systems.xtrace.XTrace;
+
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
@SuppressWarnings({"unchecked", "rawtypes"})
public class Shuffle implements ShuffleConsumerPlugin, ExceptionReporter {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(Shuffle.class);
private static final int PROGRESS_FREQUENCY = 2000;
private static final int MAX_EVENTS_TO_FETCH = 10000;
private static final int MIN_EVENTS_TO_FETCH = 100;
@@ -53,6 +56,7 @@ public class Shuffle implements ShuffleConsumerPlugin, ExceptionRepo
private MergeManager merger;
private Throwable throwable = null;
private String throwingThreadName = null;
+ private edu.brown.cs.systems.xtrace.Context throwingContext = null;
private Progress copyPhase;
private TaskStatus taskStatus;
private Task reduceTask; //Used for status updates
@@ -89,6 +93,8 @@ protected MergeManager createMergeManager(
@Override
public RawKeyValueIterator run() throws IOException, InterruptedException {
+ xtrace.log("Running shuffle");
+
// Scale the maximum events we fetch per RPC call to mitigate OOM issues
// on the ApplicationMaster when a thundering herd of reducers fetch events
// TODO: This should not be necessary after HADOOP-8942
@@ -118,23 +124,28 @@ public RawKeyValueIterator run() throws IOException, InterruptedException {
synchronized (this) {
if (throwable != null) {
+ XTrace.join(throwingContext);
+ xtrace.log("Error during shuffle: "+throwable.getClass().getName(),
+ "Throwing Thread", throwingThreadName, "Message", throwable.getMessage());
throw new ShuffleError("error in shuffle in " + throwingThreadName,
throwable);
}
}
}
-
+
// Stop the event-fetcher thread
eventFetcher.shutDown();
// Stop the map-output fetcher threads
for (Fetcher fetcher : fetchers) {
fetcher.shutDown();
+ fetcher.joinContexts();
}
// stop the scheduler
scheduler.close();
+ xtrace.log("Copy phase complete");
copyPhase.complete(); // copy is already complete
taskStatus.setPhase(TaskStatus.Phase.SORT);
reduceTask.statusUpdate(umbilical);
@@ -144,17 +155,23 @@ public RawKeyValueIterator run() throws IOException, InterruptedException {
try {
kvIter = merger.close();
} catch (Throwable e) {
+ xtrace.log("Error during final merge", "Throwable", e.getClass().getName(), "Message", e.getMessage());
throw new ShuffleError("Error while doing final merge " , e);
}
// Sanity check
synchronized (this) {
if (throwable != null) {
+ XTrace.join(throwingContext);
+ xtrace.log("Error during shuffle: "+throwable.getClass().getName(),
+ "Throwing Thread", throwingThreadName, "Message", throwable.getMessage());
throw new ShuffleError("error in shuffle in " + throwingThreadName,
throwable);
}
}
+ xtrace.log("Shuffle complete");
+
return kvIter;
}
@@ -166,6 +183,7 @@ public synchronized void reportException(Throwable t) {
if (throwable == null) {
throwable = t;
throwingThreadName = Thread.currentThread().getName();
+ throwingContext = XTrace.get();
// Notify the scheduler so that the reporting thread finds the
// exception immediately.
synchronized (scheduler) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleHeader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleHeader.java
index b42c018427dc..31c19e8e32f8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleHeader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleHeader.java
@@ -27,6 +27,10 @@
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.Metadata.XTraceMetadata;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Shuffle Header information that is sent by the TaskTracker and
* deciphered by the Fetcher thread of Reduce task
@@ -51,6 +55,7 @@ public class ShuffleHeader implements Writable {
long uncompressedLength;
long compressedLength;
int forReduce;
+ Context m;
public ShuffleHeader() { }
@@ -67,6 +72,7 @@ public void readFields(DataInput in) throws IOException {
compressedLength = WritableUtils.readVLong(in);
uncompressedLength = WritableUtils.readVLong(in);
forReduce = WritableUtils.readVInt(in);
+ XTrace.join(WritableUtils.readCompressedByteArray(in));
}
public void write(DataOutput out) throws IOException {
@@ -74,5 +80,7 @@ public void write(DataOutput out) throws IOException {
WritableUtils.writeVLong(out, compressedLength);
WritableUtils.writeVLong(out, uncompressedLength);
WritableUtils.writeVInt(out, forReduce);
+ WritableUtils.writeCompressedByteArray(out, XTrace.bytes());
}
+
}
\ No newline at end of file
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleScheduler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleScheduler.java
index 95c750930a56..e5aad8c14318 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleScheduler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleScheduler.java
@@ -44,4 +44,4 @@ public void resolve(TaskCompletionEvent tce)
public void close() throws InterruptedException;
-}
+}
\ No newline at end of file
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleSchedulerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleSchedulerImpl.java
index 76affb234be3..beedcd463422 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleSchedulerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleSchedulerImpl.java
@@ -18,10 +18,10 @@
package org.apache.hadoop.mapreduce.task.reduce;
import java.io.IOException;
-
import java.net.URI;
import java.text.DecimalFormat;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@@ -48,6 +48,9 @@
import org.apache.hadoop.mapreduce.task.reduce.MapHost.State;
import org.apache.hadoop.util.Progress;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class ShuffleSchedulerImpl implements ShuffleScheduler {
@@ -57,6 +60,7 @@ protected Long initialValue() {
}
};
+ private static final XTrace.Logger xtrace = XTrace.getLogger(ShuffleSchedulerImpl.class);
private static final Log LOG = LogFactory.getLog(ShuffleSchedulerImpl.class);
private static final int MAX_MAPS_AT_ONCE = 20;
private static final long INITIAL_PENALTY = 10000;
@@ -100,6 +104,8 @@ protected Long initialValue() {
private final boolean reportReadErrorImmediately;
private long maxDelay = MRJobConfig.DEFAULT_MAX_SHUFFLE_FETCH_RETRY_DELAY;
+ private Collection failure_contexts = new HashSet();
+
public ShuffleSchedulerImpl(JobConf job, TaskStatus status,
TaskAttemptID reduceId,
ExceptionReporter reporter,
@@ -195,6 +201,7 @@ public synchronized void copySucceeded(TaskAttemptID mapId,
reduceShuffleBytes.increment(bytes);
lastProgressTime = System.currentTimeMillis();
LOG.debug("map " + mapId + " done " + status.getStateString());
+ xtrace.log("Shuffle from mapper complete", "Map ID", mapId, "Status", status.getStateString());
}
}
@@ -230,8 +237,11 @@ public synchronized void copyFailed(TaskAttemptID mapId, MapHost host,
} else {
hostFailures.put(hostname, new IntWritable(1));
}
+ failure_contexts.add(XTrace.get());
if (failures >= abortFailureLimit) {
try {
+ for (Context ctx : failure_contexts)
+ XTrace.join(ctx);
throw new IOException(failures + " failures downloading " + mapId);
} catch (IOException ie) {
reporter.reportException(ie);
@@ -368,6 +378,8 @@ public synchronized MapHost getHost() throws InterruptedException {
LOG.info("Assigning " + host + " with " + host.getNumKnownMapOutputs() +
" to " + Thread.currentThread().getName());
+ xtrace.log("Selected a host for shuffle",
+ "Host", host, "Num Outputs", host.getNumKnownMapOutputs(), "Thread Name", Thread.currentThread().getName());
shuffleStart.set(System.currentTimeMillis());
return host;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
index 01dcbe4dc83f..7fbc7420ec1c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
@@ -450,4 +450,16 @@ public List getAMInfos() {
}
return amInfos;
}
+
+ @Override
+ public int getAndJoinCompletedMaps() {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public int getAndJoinCompletedReduces() {
+ // TODO Auto-generated method stub
+ return 0;
+ }
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
index 2c1f3a26fff6..b5523c9b9792 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
@@ -172,6 +172,7 @@ public String getApplicationName() {
}
private class MoveIntermediateToDoneRunnable implements Runnable {
+
@Override
public void run() {
try {
@@ -184,6 +185,7 @@ public void run() {
}
private class HistoryCleaner implements Runnable {
+
public void run() {
LOG.info("History Cleaner started");
try {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java
index ce51c390b151..0f0bfc2b6c0b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java
@@ -190,4 +190,15 @@ public List getAMInfos() {
return null;
}
+
+ @Override
+ public int getAndJoinCompletedMaps() {
+ return getCompletedMaps();
+ }
+
+ @Override
+ public int getAndJoinCompletedReduces() {
+ return getCompletedReduces();
+ }
+
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
index ba02d86bdef3..f723ed0cd123 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
@@ -47,8 +47,8 @@
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
import org.apache.hadoop.yarn.client.api.YarnClient;
import org.apache.hadoop.yarn.client.api.YarnClientApplication;
@@ -59,7 +59,10 @@
import com.google.common.annotations.VisibleForTesting;
+import edu.brown.cs.systems.xtrace.XTrace;
+
public class ResourceMgrDelegate extends YarnClient {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(ResourceMgrDelegate.class);
private static final Log LOG = LogFactory.getLog(ResourceMgrDelegate.class);
private YarnConfiguration conf;
@@ -170,7 +173,9 @@ public JobID getNewJobID() throws IOException, InterruptedException {
try {
this.application = client.createApplication().getApplicationSubmissionContext();
this.applicationId = this.application.getApplicationId();
- return TypeConverter.fromYarn(applicationId);
+ JobID id = TypeConverter.fromYarn(applicationId);
+ xtrace.log("Job ID acquired", "Job ID", id.getId());
+ return id;
} catch (YarnException e) {
throw new IOException(e);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
index 3d2952ba2edf..59c0631bd099 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
@@ -315,6 +315,7 @@ private LocalResource createApplicationResource(FileContext fs, Path p, LocalRes
rsrc.setTimestamp(rsrcStat.getModificationTime());
rsrc.setType(type);
rsrc.setVisibility(LocalResourceVisibility.APPLICATION);
+ rsrc.rememberContext();
return rsrc;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java
index f0840841fbd2..9c2c8229e721 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java
@@ -29,6 +29,9 @@
import org.apache.hadoop.io.nativeio.NativeIO;
import org.jboss.netty.handler.stream.ChunkedFile;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
public class FadvisedChunkedFile extends ChunkedFile {
private static final Log LOG = LogFactory.getLog(FadvisedChunkedFile.class);
@@ -40,6 +43,8 @@ public class FadvisedChunkedFile extends ChunkedFile {
private final String identifier;
private ReadaheadRequest readaheadRequest;
+
+ private Context xtrace;
public FadvisedChunkedFile(RandomAccessFile file, long position, long count,
int chunkSize, boolean manageOsCache, int readaheadLength,
@@ -50,16 +55,25 @@ public FadvisedChunkedFile(RandomAccessFile file, long position, long count,
this.readaheadPool = readaheadPool;
this.fd = file.getFD();
this.identifier = identifier;
+ this.xtrace = XTrace.get();
}
@Override
public Object nextChunk() throws Exception {
+ Context before = XTrace.get();
+ if (before==null)
+ XTrace.set(xtrace);
if (manageOsCache && readaheadPool != null) {
readaheadRequest = readaheadPool
.readaheadStream(identifier, fd, getCurrentOffset(), readaheadLength,
getEndOffset(), readaheadRequest);
}
- return super.nextChunk();
+ try {
+ return super.nextChunk();
+ } finally {
+ if (before==null)
+ XTrace.stop();
+ }
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java
index 9bb3fb0180a1..023911c71ebc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java
@@ -30,6 +30,9 @@
import org.apache.hadoop.io.nativeio.NativeIO;
import org.jboss.netty.channel.DefaultFileRegion;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
public class FadvisedFileRegion extends DefaultFileRegion {
private static final Log LOG = LogFactory.getLog(FadvisedFileRegion.class);
@@ -41,6 +44,8 @@ public class FadvisedFileRegion extends DefaultFileRegion {
private final String identifier;
private ReadaheadRequest readaheadRequest;
+
+ private Context xtrace;
public FadvisedFileRegion(RandomAccessFile file, long position, long count,
boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool,
@@ -51,17 +56,26 @@ public FadvisedFileRegion(RandomAccessFile file, long position, long count,
this.readaheadPool = readaheadPool;
this.fd = file.getFD();
this.identifier = identifier;
+ this.xtrace = XTrace.get();
}
@Override
public long transferTo(WritableByteChannel target, long position)
throws IOException {
+ Context before = XTrace.get();
+ if (before==null)
+ XTrace.set(xtrace);
if (manageOsCache && readaheadPool != null) {
readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
getPosition() + position, readaheadLength,
getPosition() + getCount(), readaheadRequest);
}
- return super.transferTo(target, position);
+ try {
+ return super.transferTo(target, position);
+ } finally {
+ if (before==null)
+ XTrace.stop();
+ }
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
index f9d1203ba4df..8613e1b6cab8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
@@ -23,7 +23,6 @@
import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.HTTP_VERSION_NOT_SUPPORTED;
import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
@@ -114,9 +113,17 @@
import com.google.common.base.Charsets;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import edu.brown.cs.systems.resourcethrottling.LocalThrottlingPoints;
+import edu.brown.cs.systems.resourcethrottling.ThrottlingPoint;
+import edu.brown.cs.systems.resourcetracing.resources.Network;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
public class ShuffleHandler extends AuxiliaryService {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(ShuffleHandler.class);
private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
+ private static final ThrottlingPoint shuffle_throttler = LocalThrottlingPoints.getThrottlingPoint("ShuffleHandler");
public static final String SHUFFLE_MANAGE_OS_CACHE = "mapreduce.shuffle.manage.os.cache";
public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
@@ -446,6 +453,11 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
request.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) {
sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST);
}
+
+ String xtrace_context = request.getHeader("X-Trace");
+ if (xtrace_context!=null)
+ XTrace.set(xtrace_context, XTrace.ENCODING.BASE16);
+
final Map> q =
new QueryStringDecoder(request.getUri()).getParameters();
final List mapIds = splitMaps(q.get("map"));
@@ -457,6 +469,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
"\n reduceId: " + reduceQ +
"\n jobId: " + jobQ);
}
+ xtrace.log("Handling map output retrieval request", "URI", request.getUri(), "Map IDs", mapIds,
+ "Reduce ID", reduceQ, "Job ID", jobQ);
if (mapIds == null || reduceQ == null || jobQ == null) {
sendError(ctx, "Required param job, map and reduce", BAD_REQUEST);
@@ -498,11 +512,13 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
ch.write(response);
// TODO refactor the following into the pipeline
ChannelFuture lastMap = null;
+ Context start_context = XTrace.get();
for (String mapId : mapIds) {
try {
lastMap =
sendMapOutput(ctx, ch, userRsrc.get(jobId), jobId, mapId, reduceId);
if (null == lastMap) {
+ xtrace.log("Error: "+NOT_FOUND);
sendError(ctx, NOT_FOUND);
return;
}
@@ -514,9 +530,11 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
sb.append(t.getCause().getMessage());
t = t.getCause();
}
+ xtrace.log("Shuffle error: "+e.getClass().getName(), "Message", e.getMessage());
sendError(ctx,sb.toString() , INTERNAL_SERVER_ERROR);
return;
}
+ XTrace.set(start_context);
}
lastMap.addListener(metrics);
lastMap.addListener(ChannelFutureListener.CLOSE);
@@ -566,6 +584,8 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
String user, String jobId, String mapId, int reduce)
throws IOException {
+ shuffle_throttler.throttle();
+
// TODO replace w/ rsrc alloc
// $x/$user/appcache/$appId/output/$mapId
// TODO: Once Shuffle is out of NM, this can use MR APIs to convert between App and Job
@@ -591,6 +611,8 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
}
final IndexRecord info =
indexCache.getIndexInformation(mapId, reduce, indexFileName, user);
+ info.joinContext();
+ xtrace.log("Sending map output", "Reduce", reduce, "Map ID", mapId, "Job ID", jobID);
final ShuffleHeader header =
new ShuffleHeader(mapId, info.partLength, info.rawLength, reduce);
final DataOutputBuffer dob = new DataOutputBuffer();
@@ -647,6 +669,7 @@ protected void sendError(ChannelHandlerContext ctx, String message,
ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
response.setContent(
ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
+ response.setHeader("X-Trace", XTrace.base64());
// Close the connection as soon as the error message is sent.
ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordCount.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordCount.java
index 48c02fe1e118..72486484b982 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordCount.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordCount.java
@@ -64,12 +64,12 @@ public void reduce(Text key, Iterable values,
context.write(key, result);
}
}
-
+
public static void main(String[] args) throws Exception {
Configuration conf = new Configuration();
String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
- if (otherArgs.length != 2) {
- System.err.println("Usage: wordcount ");
+ if (otherArgs.length < 2 || otherArgs.length > 3) {
+ System.err.println("Usage: wordcount ");
System.exit(2);
}
Job job = new Job(conf, "word count");
@@ -81,6 +81,10 @@ public static void main(String[] args) throws Exception {
job.setOutputValueClass(IntWritable.class);
FileInputFormat.addInputPath(job, new Path(otherArgs[0]));
FileOutputFormat.setOutputPath(job, new Path(otherArgs[1]));
+ if (otherArgs.length==3) {
+ int numred = otherArgs.length > 2 ? Integer.valueOf(otherArgs[2]) : 2;
+ job.setNumReduceTasks(numred);
+ }
System.exit(job.waitForCompletion(true) ? 0 : 1);
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
index ccbe6f161030..f4924c14a57f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
@@ -41,6 +41,7 @@ public static ContainerId newInstance(ApplicationAttemptId appAttemptId,
ContainerId id = Records.newRecord(ContainerId.class);
id.setId(containerId);
id.setApplicationAttemptId(appAttemptId);
+ id.rememberContext();
id.build();
return id;
}
@@ -70,6 +71,9 @@ public static ContainerId newInstance(ApplicationAttemptId appAttemptId,
@Private
@Unstable
protected abstract void setId(int id);
+
+ public abstract void rememberContext();
+ public abstract void joinContext();
// TODO: fail the app submission if attempts are more than 10 or something
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java
index f14a136d30d7..0631543ebe53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java
@@ -170,4 +170,15 @@ public static LocalResource newInstance(URL url, LocalResourceType type,
@Public
@Stable
public abstract void setPattern(String pattern);
+
+ /**
+ * XTrace - attach an xtrace context to this resource, will be serialized and sent over the wire
+ */
+ public abstract void rememberContext();
+
+ /**
+ * XTrace - join any contexts that are part of this message
+ */
+ public abstract void joinContext();
+
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
index d7aa413b8e23..bb07122ec92f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
@@ -239,6 +239,9 @@ public static boolean isAnyLocation(String hostName) {
@Stable
public abstract void setRelaxLocality(boolean relaxLocality);
+ public abstract void rememberContext();
+ public abstract void joinContext();
+
@Override
public int hashCode() {
final int prime = 2153;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index a27cbb7cd518..620a1d9cb042 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -50,6 +50,7 @@ message ContainerIdProto {
optional ApplicationIdProto app_id = 1;
optional ApplicationAttemptIdProto app_attempt_id = 2;
optional int32 id = 3;
+ optional bytes xtrace = 4; // X-Trace metadata
}
message ResourceProto {
@@ -120,6 +121,7 @@ message LocalResourceProto {
optional LocalResourceTypeProto type = 4;
optional LocalResourceVisibilityProto visibility = 5;
optional string pattern = 6;
+ optional bytes xtrace = 7; // X-Trace metadata
}
message ApplicationResourceUsageReportProto {
@@ -188,6 +190,7 @@ message ResourceRequestProto {
optional ResourceProto capability = 3;
optional int32 num_containers = 4;
optional bool relax_locality = 5 [default = true];
+ optional bytes xtrace = 6; // X-Trace metadata
}
enum AMCommandProto {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/YarnUncaughtExceptionHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/YarnUncaughtExceptionHandler.java
index 542fb228801f..1701e8078570 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/YarnUncaughtExceptionHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/YarnUncaughtExceptionHandler.java
@@ -27,6 +27,8 @@
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.ShutdownHookManager;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* This class is intended to be installed by calling
* {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)}
@@ -39,6 +41,7 @@
@Public
@Evolving
public class YarnUncaughtExceptionHandler implements UncaughtExceptionHandler {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(YarnUncaughtExceptionHandler.class);
private static final Log LOG = LogFactory.getLog(YarnUncaughtExceptionHandler.class);
@Override
@@ -49,6 +52,8 @@ public void uncaughtException(Thread t, Throwable e) {
} else if(e instanceof Error) {
try {
LOG.fatal("Thread " + t + " threw an Error. Shutting down now...", e);
+ xtrace.log(e.getClass().getName(),
+ "Thread Name", t.getName(), "Message", e.getMessage());
} catch (Throwable err) {
//We don't want to not exit because of an issue with logging
}
@@ -57,15 +62,20 @@ public void uncaughtException(Thread t, Throwable e) {
//even try to clean up or we can get stuck on shutdown.
try {
System.err.println("Halting due to Out Of Memory Error...");
+// // TODO: XTrace do this
+// XTraceContext.joinParentProcess();
} catch (Throwable err) {
//Again we done want to exit because of logging issues.
}
ExitUtil.halt(-1);
} else {
+// // TODO: XTrace do this
+// XTraceContext.joinParentProcess();
ExitUtil.terminate(-1);
}
} else {
LOG.error("Thread " + t + " threw an Exception.", e);
+ xtrace.log(e.getClass().getName(), "Thread Name", t.getName(), "Message", e.getMessage());
}
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java
index 9be829fb490d..634b17e38e8b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java
@@ -26,6 +26,10 @@
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
@Private
@Unstable
@@ -33,6 +37,7 @@ public class ContainerIdPBImpl extends ContainerId {
ContainerIdProto proto = null;
ContainerIdProto.Builder builder = null;
private ApplicationAttemptId applicationAttemptId = null;
+ Context xmd = null;
public ContainerIdPBImpl() {
builder = ContainerIdProto.newBuilder();
@@ -41,6 +46,9 @@ public ContainerIdPBImpl() {
public ContainerIdPBImpl(ContainerIdProto proto) {
this.proto = proto;
this.applicationAttemptId = convertFromProtoFormat(proto.getAppAttemptId());
+ if (proto!=null && proto.hasXtrace()) {
+ xmd = Context.parse(proto.getXtrace().toByteArray());
+ }
}
public ContainerIdProto getProto() {
@@ -74,6 +82,16 @@ protected void setApplicationAttemptId(ApplicationAttemptId atId) {
this.applicationAttemptId = atId;
}
+ @Override
+ public void rememberContext() {
+ xmd = XTrace.get();
+ }
+
+ @Override
+ public void joinContext() {
+ XTrace.join(xmd);
+ }
+
private ApplicationAttemptIdPBImpl convertFromProtoFormat(
ApplicationAttemptIdProto p) {
return new ApplicationAttemptIdPBImpl(p);
@@ -86,6 +104,9 @@ private ApplicationAttemptIdProto convertToProtoFormat(
@Override
protected void build() {
+ if (xmd!=null) {
+ builder.setXtrace(ByteString.copyFrom(xmd.bytes()));
+ }
proto = builder.build();
builder = null;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LocalResourcePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LocalResourcePBImpl.java
index 16bd59740d20..8d3341da1040 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LocalResourcePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LocalResourcePBImpl.java
@@ -30,8 +30,11 @@
import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceVisibilityProto;
import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
+import com.google.protobuf.ByteString;
import com.google.protobuf.TextFormat;
+import edu.brown.cs.systems.xtrace.XTrace;
+
@Private
@Unstable
public class LocalResourcePBImpl extends LocalResource {
@@ -192,6 +195,20 @@ public synchronized void setPattern(String pattern) {
builder.setPattern(pattern);
}
+ @Override
+ public void rememberContext() {
+ maybeInitBuilder();
+ if (XTrace.active())
+ builder.setXtrace(ByteString.copyFrom(XTrace.bytes()));
+ }
+
+ @Override
+ public void joinContext() {
+ LocalResourceProtoOrBuilder p = viaProto ? proto : builder;
+ if (p.hasXtrace())
+ XTrace.join(p.getXtrace().toByteArray());
+ }
+
private LocalResourceTypeProto convertToProtoFormat(LocalResourceType e) {
return ProtoUtils.convertToProtoFormat(e);
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
index 22863ac551b8..98c631cf074b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
@@ -29,6 +29,10 @@
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProtoOrBuilder;
+import com.google.protobuf.ByteString;
+
+import edu.brown.cs.systems.xtrace.XTrace;
+
@Private
@Unstable
public class ResourceRequestPBImpl extends ResourceRequest {
@@ -163,6 +167,20 @@ public void setRelaxLocality(boolean relaxLocality) {
builder.setRelaxLocality(relaxLocality);
}
+ @Override
+ public void rememberContext() {
+ maybeInitBuilder();
+ if (XTrace.active())
+ builder.setXtrace(ByteString.copyFrom(XTrace.bytes()));
+ }
+
+ @Override
+ public void joinContext() {
+ ResourceRequestProtoOrBuilder p = viaProto ? proto : builder;
+ if (p.hasXtrace())
+ XTrace.join(p.getXtrace().toByteArray());
+ }
+
private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
return new PriorityPBImpl(p);
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AbstractEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AbstractEvent.java
index 5ff0e1078d38..659c0f3f4ab5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AbstractEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AbstractEvent.java
@@ -21,6 +21,9 @@
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Parent class of all the events. All events extend this class.
*/
@@ -31,12 +34,14 @@ public abstract class AbstractEvent>
private final TYPE type;
private final long timestamp;
+ private Context xtraceContext;
// use this if you DON'T care about the timestamp
public AbstractEvent(TYPE type) {
this.type = type;
// We're not generating a real timestamp here. It's too expensive.
timestamp = -1L;
+ this.rememberContext();
}
// use this if you care about the timestamp
@@ -59,4 +64,18 @@ public TYPE getType() {
public String toString() {
return "EventType: " + getType();
}
+
+ /*
+ * XTrace methods
+ */
+ @Override
+ public void rememberContext() {
+ this.xtraceContext = XTrace.get();
+ }
+
+ @Override
+ public void joinContext() {
+ XTrace.join(this.xtraceContext);
+ }
+
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
index 9301bba081b4..30e0b63cff3e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
@@ -19,7 +19,9 @@
package org.apache.hadoop.yarn.event;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
@@ -34,6 +36,9 @@
import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Dispatches {@link Event}s in a separate thread. Currently only single thread
* does that. Potentially there could be multiple channels for each event type
@@ -44,6 +49,7 @@
@Evolving
public class AsyncDispatcher extends AbstractService implements Dispatcher {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(AsyncDispatcher.class);
private static final Log LOG = LogFactory.getLog(AsyncDispatcher.class);
private final BlockingQueue eventQueue;
@@ -68,6 +74,7 @@ Runnable createThread() {
@Override
public void run() {
while (!stopped && !Thread.currentThread().isInterrupted()) {
+ XTrace.stop();
Event event;
try {
event = eventQueue.take();
@@ -78,6 +85,7 @@ public void run() {
return;
}
if (event != null) {
+ event.joinContext();
dispatch(event);
}
}
@@ -120,6 +128,7 @@ protected void serviceStop() throws Exception {
@SuppressWarnings("unchecked")
protected void dispatch(Event event) {
+ event.joinContext();
//all events go thru this loop
if (LOG.isDebugEnabled()) {
LOG.debug("Dispatching the event " + event.getClass().getName() + "."
@@ -142,6 +151,9 @@ protected void dispatch(Event event) {
if (exitOnDispatchException
&& (ShutdownHookManager.get().isShutdownInProgress()) == false) {
LOG.info("Exiting, bbye..");
+ xtrace.log("Exiting, bbye..");
+// // TODO: XTrace deal with this
+// XTraceContext.joinParentProcess();
System.exit(-1);
}
}
@@ -213,9 +225,16 @@ public MultiListenerHandler() {
@Override
public void handle(Event event) {
+ Context start_xtrace_context = XTrace.get();
+ Collection result_xtrace_contexts = new HashSet();
for (EventHandler handler: listofHandlers) {
+ XTrace.set(start_xtrace_context);
+ event.joinContext();
handler.handle(event);
+ result_xtrace_contexts.add(XTrace.get());
}
+ for (Context ctx : result_xtrace_contexts)
+ XTrace.join(ctx);
}
void addHandler(EventHandler handler) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/Event.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/Event.java
index e76a7530a7ec..a177f880da07 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/Event.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/Event.java
@@ -32,4 +32,6 @@ public interface Event> {
TYPE getType();
long getTimestamp();
String toString();
+ void rememberContext();
+ void joinContext();
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java
index b5ce6a37c15b..91ff58fb6e04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java
@@ -29,4 +29,5 @@ public interface StateMachine
public STATE getCurrentState();
public STATE doTransition(EVENTTYPE eventType, EVENT event)
throws InvalidStateTransitonException;
+ public void joinPreviousTransitionXTraceContext();
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
index 55ac4cf22223..9df93cb502ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
@@ -29,6 +29,9 @@
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* State machine topology.
* This object is semantically immutable. If you have a
@@ -46,6 +49,8 @@
final public class StateMachineFactory
,
EVENTTYPE extends Enum, EVENT> {
+
+ public enum Trace { NEVER, ALWAYS, KEEPALIVE, IGNORE }
private final TransitionsListNode transitionsListNode;
@@ -55,18 +60,31 @@ final public class StateMachineFactory
private STATE defaultInitialState;
private final boolean optimized;
+
+ /**
+ * X-Trace variables
+ */
+ private int xtrace_id_seed = 1;
+ private Trace xtrace_default = Trace.ALWAYS;
/**
- * Constructor
- *
- * This is the only constructor in the API.
+ * Constructor (Default MR)
*
*/
public StateMachineFactory(STATE defaultInitialState) {
+ this(defaultInitialState, Trace.ALWAYS);
+ }
+
+ /**
+ * Constructor (X-Trace)
+ *
+ */
+ public StateMachineFactory(STATE defaultInitialState, Trace xtrace_default) {
this.transitionsListNode = null;
this.defaultInitialState = defaultInitialState;
this.optimized = false;
this.stateMachineTable = null;
+ this.xtrace_default = xtrace_default;
}
private StateMachineFactory
@@ -77,6 +95,7 @@ public StateMachineFactory(STATE defaultInitialState) {
= new TransitionsListNode(t, that.transitionsListNode);
this.optimized = false;
this.stateMachineTable = null;
+ this.xtrace_default = that.xtrace_default;
}
private StateMachineFactory
@@ -90,6 +109,7 @@ public StateMachineFactory(STATE defaultInitialState) {
} else {
stateMachineTable = null;
}
+ this.xtrace_default = that.xtrace_default;
}
private interface ApplicableTransition
@@ -160,7 +180,28 @@ static private class ApplicableSingleOrMultipleTransition
public StateMachineFactory
addTransition(STATE preState, STATE postState, EVENTTYPE eventType) {
- return addTransition(preState, postState, eventType, null);
+ return addTransition(preState, postState, eventType, xtrace_default);
+ }
+
+ /**
+ * @return a NEW StateMachineFactory just like {@code this} with the current
+ * transition added as a new legal transition. This overload
+ * has no hook object.
+ *
+ * Note that the returned StateMachineFactory is a distinct
+ * object.
+ *
+ * This method is part of the API.
+ *
+ * @param preState pre-transition state
+ * @param postState post-transition state
+ * @param eventType stimulus for the transition
+ * @param xtrace_process boolean flag, indicating whether the transition should be logged as an xtrace process with an edge to the previous transition (or not)
+ */
+ public StateMachineFactory
+
+ addTransition(STATE preState, STATE postState, EVENTTYPE eventType, Trace xtrace_process) {
+ return addTransition(preState, postState, eventType, null, xtrace_process);
}
/**
@@ -180,7 +221,28 @@ static private class ApplicableSingleOrMultipleTransition
*/
public StateMachineFactory addTransition(
STATE preState, STATE postState, Set eventTypes) {
- return addTransition(preState, postState, eventTypes, null);
+ return addTransition(preState, postState, eventTypes, xtrace_default);
+ }
+
+ /**
+ * @return a NEW StateMachineFactory just like {@code this} with the current
+ * transition added as a new legal transition. This overload
+ * has no hook object.
+ *
+ *
+ * Note that the returned StateMachineFactory is a distinct
+ * object.
+ *
+ * This method is part of the API.
+ *
+ * @param preState pre-transition state
+ * @param postState post-transition state
+ * @param eventTypes List of stimuli for the transitions
+ * @param xtrace_process boolean flag, indicating whether the transition should be logged as an xtrace process with an edge to the previous transition (or not)
+ */
+ public StateMachineFactory addTransition(
+ STATE preState, STATE postState, Set eventTypes, Trace xtrace_process) {
+ return addTransition(preState, postState, eventTypes, null, xtrace_process);
}
/**
@@ -200,12 +262,33 @@ public StateMachineFactory addTransition(
public StateMachineFactory addTransition(
STATE preState, STATE postState, Set eventTypes,
SingleArcTransition hook) {
+ return addTransition(preState, postState, eventTypes, hook, xtrace_default);
+ }
+
+ /**
+ * @return a NEW StateMachineFactory just like {@code this} with the current
+ * transition added as a new legal transition
+ *
+ * Note that the returned StateMachineFactory is a distinct
+ * object.
+ *
+ * This method is part of the API.
+ *
+ * @param preState pre-transition state
+ * @param postState post-transition state
+ * @param eventTypes List of stimuli for the transitions
+ * @param hook transition hook
+ * @param xtrace_process boolean flag, indicating whether the transition should be logged as an xtrace process with an edge to the previous transition (or not)
+ */
+ public StateMachineFactory addTransition(
+ STATE preState, STATE postState, Set eventTypes,
+ SingleArcTransition hook, Trace xtrace_process) {
StateMachineFactory factory = null;
for (EVENTTYPE event : eventTypes) {
if (factory == null) {
- factory = addTransition(preState, postState, event, hook);
+ factory = addTransition(preState, postState, event, hook, xtrace_process);
} else {
- factory = factory.addTransition(preState, postState, event, hook);
+ factory = factory.addTransition(preState, postState, event, hook, xtrace_process);
}
}
return factory;
@@ -229,9 +312,31 @@ public StateMachineFactory addTransition(
addTransition(STATE preState, STATE postState,
EVENTTYPE eventType,
SingleArcTransition hook){
+ return addTransition(preState, postState, eventType, hook, xtrace_default);
+ }
+
+ /**
+ * @return a NEW StateMachineFactory just like {@code this} with the current
+ * transition added as a new legal transition
+ *
+ * Note that the returned StateMachineFactory is a distinct object.
+ *
+ * This method is part of the API.
+ *
+ * @param preState pre-transition state
+ * @param postState post-transition state
+ * @param eventType stimulus for the transition
+ * @param hook transition hook
+ * @param xtrace_process boolean flag, indicating whether the transition should be logged as an xtrace process with an edge to the previous transition (or not)
+ */
+ public StateMachineFactory
+
+ addTransition(STATE preState, STATE postState,
+ EVENTTYPE eventType,
+ SingleArcTransition hook, Trace xtrace_process){
return new StateMachineFactory
(this, new ApplicableSingleOrMultipleTransition
- (preState, eventType, new SingleInternalArc(postState, hook)));
+ (preState, eventType, new SingleInternalArc(postState, hook, xtrace_process)));
}
/**
@@ -252,10 +357,32 @@ public StateMachineFactory addTransition(
addTransition(STATE preState, Set postStates,
EVENTTYPE eventType,
MultipleArcTransition hook){
+ return addTransition(preState, postStates, eventType, hook, xtrace_default);
+ }
+
+ /**
+ * @return a NEW StateMachineFactory just like {@code this} with the current
+ * transition added as a new legal transition
+ *
+ * Note that the returned StateMachineFactory is a distinct object.
+ *
+ * This method is part of the API.
+ *
+ * @param preState pre-transition state
+ * @param postStates valid post-transition states
+ * @param eventType stimulus for the transition
+ * @param hook transition hook
+ * @param xtrace_process boolean flag, indicating whether the transition should be logged as an xtrace process with an edge to the previous transition (or not)
+ */
+ public StateMachineFactory
+
+ addTransition(STATE preState, Set postStates,
+ EVENTTYPE eventType,
+ MultipleArcTransition hook, Trace xtrace_process){
return new StateMachineFactory
(this,
new ApplicableSingleOrMultipleTransition
- (preState, eventType, new MultipleInternalArc(postStates, hook)));
+ (preState, eventType, new MultipleInternalArc(postStates, hook, xtrace_process)));
}
/**
@@ -304,6 +431,40 @@ public StateMachineFactory addTransition(
}
throw new InvalidStateTransitonException(oldState, eventType);
}
+
+ private boolean logXTraceTransition(STATE oldState, EVENTTYPE eventType) {
+ Map> transitionMap
+ = stateMachineTable.get(oldState);
+ if (transitionMap != null) {
+ Transition transition
+ = transitionMap.get(eventType);
+ if (transition != null) {
+ Trace trace_type = transition.getTraceType();
+ return trace_type!=Trace.IGNORE;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Tells us whether the transition should link back to the previous transition event
+ * @param oldState current state
+ * @param eventType trigger to initiate the transition
+ * @return true if the transition should link the previous xtrace context with this one
+ */
+ private boolean logTransitionAsXTraceProcess(STATE oldState, EVENTTYPE eventType) {
+ Map> transitionMap
+ = stateMachineTable.get(oldState);
+ if (transitionMap != null) {
+ Transition transition
+ = transitionMap.get(eventType);
+ if (transition != null) {
+ Trace trace_type = transition.getTraceType();
+ return trace_type==Trace.ALWAYS || (trace_type==Trace.KEEPALIVE && !XTrace.active());
+ }
+ }
+ return false;
+ }
private synchronized void maybeMakeStateMachineTable() {
if (stateMachineTable == null) {
@@ -341,6 +502,8 @@ private interface Transition,
EVENTTYPE extends Enum, EVENT> {
STATE doTransition(OPERAND operand, STATE oldState,
EVENT event, EVENTTYPE eventType);
+
+ Trace getTraceType();
}
private class SingleInternalArc
@@ -348,11 +511,14 @@ private class SingleInternalArc
private STATE postState;
private SingleArcTransition hook; // transition hook
+ private Trace xtrace_process;
SingleInternalArc(STATE postState,
- SingleArcTransition hook) {
+ SingleArcTransition hook,
+ Trace xtrace_process) {
this.postState = postState;
this.hook = hook;
+ this.xtrace_process = xtrace_process;
}
@Override
@@ -363,6 +529,11 @@ public STATE doTransition(OPERAND operand, STATE oldState,
}
return postState;
}
+
+ @Override
+ public Trace getTraceType() {
+ return xtrace_process;
+ }
}
private class MultipleInternalArc
@@ -371,11 +542,14 @@ private class MultipleInternalArc
// Fields
private Set validPostStates;
private MultipleArcTransition hook; // transition hook
+ private Trace xtrace_process;
MultipleInternalArc(Set postStates,
- MultipleArcTransition hook) {
+ MultipleArcTransition hook,
+ Trace xtrace_process) {
this.validPostStates = postStates;
this.hook = hook;
+ this.xtrace_process = xtrace_process;
}
@Override
@@ -389,6 +563,11 @@ public STATE doTransition(OPERAND operand, STATE oldState,
}
return postState;
}
+
+ @Override
+ public Trace getTraceType() {
+ return xtrace_process;
+ }
}
/*
@@ -406,7 +585,7 @@ public STATE doTransition(OPERAND operand, STATE oldState,
*/
public StateMachine
make(OPERAND operand, STATE initialState) {
- return new InternalStateMachine(operand, initialState);
+ return new InternalStateMachine(operand, initialState, xtrace_id_seed++);
}
/*
@@ -421,32 +600,71 @@ public STATE doTransition(OPERAND operand, STATE oldState,
*
*/
public StateMachine make(OPERAND operand) {
- return new InternalStateMachine(operand, defaultInitialState);
+ return new InternalStateMachine(operand, defaultInitialState, xtrace_id_seed++);
+ }
+
+ private String xtraceStateName(STATE state) {
+ return state.getDeclaringClass().getSimpleName() + "." + state.name();
}
private class InternalStateMachine
implements StateMachine {
+ private final XTrace.Logger xtrace;
private final OPERAND operand;
private STATE currentState;
- InternalStateMachine(OPERAND operand, STATE initialState) {
+ private String state_machine_id;
+ private Context previous_transition_context;
+
+ InternalStateMachine(OPERAND operand, STATE initialState, int xtrace_id_seed) {
this.operand = operand;
+ xtrace = XTrace.getLogger(operand.getClass());
this.currentState = initialState;
if (!optimized) {
maybeMakeStateMachineTable();
}
+ this.state_machine_id = operand.getClass().getSimpleName()+"-"+xtrace_id_seed;
+ xtrace.log(operand.getClass().getSimpleName() + " StateMachine initialized",
+ "StartState", xtraceStateName(currentState), "StateMachineID", state_machine_id);
+ this.previous_transition_context = XTrace.get();
}
@Override
public synchronized STATE getCurrentState() {
return currentState;
}
-
- @Override
+
+ public void joinPreviousTransitionXTraceContext() {
+ XTrace.join(this.previous_transition_context);
+ }
+
public synchronized STATE doTransition(EVENTTYPE eventType, EVENT event)
- throws InvalidStateTransitonException {
- currentState = StateMachineFactory.this.doTransition
- (operand, currentState, eventType, event);
+ throws InvalidStateTransitonException {
+ // Add an edge to the previous state machine transition if the transition is to be treated as an xtrace process
+ boolean set_previous_transition_context = false;
+ if (StateMachineFactory.this.logXTraceTransition(currentState, eventType)) {
+ if (StateMachineFactory.this.logTransitionAsXTraceProcess(currentState, eventType)) {
+ joinPreviousTransitionXTraceContext();
+ }
+
+ // Create an event for the transition
+ xtrace.log(operand.getClass().getSimpleName()+event.toString(),
+ "StartState", xtraceStateName(currentState), "Operand", operand.toString(), "StateMachineID", state_machine_id);
+
+ set_previous_transition_context = true;
+ }
+
+ // Do the transition
+ try {
+ currentState = StateMachineFactory.this.doTransition(operand, currentState, eventType, event);
+ } catch (InvalidStateTransitonException e) {
+ throw e;
+ }
+
+ if (set_previous_transition_context && XTrace.active()) {
+ this.previous_transition_context = XTrace.get();
+ }
+
return currentState;
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
index 028276b7fe0a..df94fd6682d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
@@ -43,6 +43,9 @@
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* Download a single URL to the local disk.
*
@@ -50,6 +53,7 @@
@LimitedPrivate({"YARN", "MapReduce"})
public class FSDownload implements Callable {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(FSDownload.class);
private static final Log LOG = LogFactory.getLog(FSDownload.class);
private FileContext files;
@@ -60,6 +64,8 @@ public class FSDownload implements Callable {
/** The local FS dir path under which this resource is to be localized to */
private Path destDirPath;
+ private Context xtrace_context;
+
private static final FsPermission cachePerms = new FsPermission(
(short) 0755);
static final FsPermission PUBLIC_FILE_PERMS = new FsPermission((short) 0555);
@@ -76,6 +82,7 @@ public FSDownload(FileContext files, UserGroupInformation ugi, Configuration con
this.files = files;
this.userUgi = ugi;
this.resource = resource;
+ this.xtrace_context = XTrace.get();
}
LocalResource getResource() {
@@ -169,6 +176,7 @@ private static boolean checkPermissionOfOther(FileSystem fs, Path path,
private Path copy(Path sCopy, Path dstdir) throws IOException {
+ xtrace.log("Copying", "Source", sCopy.toString(), "Destination", dstdir.toString());
FileSystem sourceFs = sCopy.getFileSystem(conf);
Path dCopy = new Path(dstdir, sCopy.getName() + ".tmp");
FileStatus sStat = sourceFs.getFileStatus(sCopy);
@@ -190,6 +198,7 @@ private Path copy(Path sCopy, Path dstdir) throws IOException {
}
private long unpack(File localrsrc, File dst, Pattern pattern) throws IOException {
+ xtrace.log("Unpacking", "Resource", localrsrc.toString(), "Unpack To", dst.toString());
switch (resource.getType()) {
case ARCHIVE: {
String lowerDst = dst.getName().toLowerCase();
@@ -262,10 +271,13 @@ private long unpack(File localrsrc, File dst, Pattern pattern) throws IOExceptio
@Override
public Path call() throws Exception {
+ XTrace.set(xtrace_context);
+ xtrace.log("Localizing resource", "Resource",resource.getResource().toString());
final Path sCopy;
try {
sCopy = ConverterUtils.getPathFromYarnURL(resource.getResource());
} catch (URISyntaxException e) {
+ xtrace.log("Invalid resource: "+e.getClass().getName(), "Message", e.getMessage());
throw new IOException("Invalid resource", e);
}
createDir(destDirPath, cachePerms);
@@ -301,7 +313,11 @@ public Path run() throws Exception {
conf = null;
resource = null;
}
- return files.makeQualified(new Path(destDirPath, sCopy.getName()));
+ Path p = files.makeQualified(new Path(destDirPath, sCopy.getName()));
+// // TODO: deal with this properly
+// XTraceContext.rememberObject(p);
+ XTrace.stop();
+ return p;
}
/**
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
index 1b2a03e55110..be05de9e81e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
@@ -101,6 +101,7 @@ public static LocalResource newLocalResource(URL url, LocalResourceType type,
resource.setVisibility(visibility);
resource.setSize(size);
resource.setTimestamp(timestamp);
+ resource.rememberContext();
return resource;
}
@@ -292,6 +293,7 @@ public static ResourceRequest newResourceRequest(Priority priority,
request.setResourceName(hostName);
request.setCapability(capability);
request.setNumContainers(numContainers);
+ request.rememberContext();
return request;
}
@@ -302,6 +304,7 @@ public static ResourceRequest newResourceRequest(ResourceRequest r) {
request.setResourceName(r.getResourceName());
request.setCapability(r.getCapability());
request.setNumContainers(r.getNumContainers());
+ //request.rememberContext();
return request;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
index 348714564feb..b880fc416b98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
@@ -53,8 +53,11 @@
import com.google.common.annotations.VisibleForTesting;
+import edu.brown.cs.systems.xtrace.XTrace;
+
public class DefaultContainerExecutor extends ContainerExecutor {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(DefaultContainerExecutor.class);
private static final Log LOG = LogFactory
.getLog(DefaultContainerExecutor.class);
@@ -187,20 +190,23 @@ public int launchContainer(Container container,
containerIdStr, this.getConf());
LOG.info("launchContainer: " + Arrays.toString(command));
+ xtrace.log("Invoking command line", "args", Arrays.toString(command), "Container ID", containerIdStr);
shExec = new ShellCommandExecutor(
command,
new File(containerWorkDir.toUri().getPath()),
container.getLaunchContext().getEnvironment()); // sanitized env
if (isContainerActive(containerId)) {
shExec.execute();
- }
- else {
+ xtrace.log("Subprocess finished with exit code "+shExec.getExitCode());
+ } else {
LOG.info("Container " + containerIdStr +
" was marked as inactive. Returning terminated error");
+ xtrace.log("Container was marked as inactive; returning terminated error");
return ExitCode.TERMINATED.getExitCode();
}
} catch (IOException e) {
if (null == shExec) {
+ xtrace.log("DefaultContainerExecutor IOException: " + e.getClass().getName(), "Message", e.getMessage());
return -1;
}
int exitCode = shExec.getExitCode();
@@ -221,6 +227,7 @@ public int launchContainer(Container container,
container.handle(new ContainerDiagnosticsUpdateEvent(containerId,
"Container killed on request. Exit code is " + exitCode));
}
+ xtrace.log("Subprocess finished with exit code "+exitCode);
return exitCode;
} finally {
; //
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
index 45504fdcdd71..11374fa41a4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
@@ -149,7 +149,7 @@ public static class FileDeletionTask implements Runnable {
// the dependent task fails then it will be marked as false in
// fileDeletionTaskFinished().
private boolean success;
-
+
private FileDeletionTask(DeletionService delService, String user,
Path subDir, List baseDirs) {
this.delService = delService;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
index 9381268b7765..d830b0ee7fd1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
@@ -43,8 +43,11 @@
import org.apache.hadoop.yarn.server.nodemanager.util.LCEResourcesHandler;
import org.apache.hadoop.yarn.util.ConverterUtils;
+import edu.brown.cs.systems.xtrace.XTrace;
+
public class LinuxContainerExecutor extends ContainerExecutor {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(LinuxContainerExecutor.class);
private static final Log LOG = LogFactory
.getLog(LinuxContainerExecutor.class);
@@ -143,6 +146,7 @@ public void init() throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("checkLinuxExecutorSetup: " + Arrays.toString(commandArray));
}
+ xtrace.log("LinuxContainerExecutor init", "args", Arrays.toString(commandArray));
try {
shExec.execute();
} catch (ExitCodeException e) {
@@ -198,6 +202,7 @@ public void startLocalizer(Path nmPrivateContainerTokensPath,
if (LOG.isDebugEnabled()) {
LOG.debug("initApplication: " + Arrays.toString(commandArray));
}
+ xtrace.log("Starting localizer", "args", Arrays.toString(commandArray));
try {
shExec.execute();
if (LOG.isDebugEnabled()) {
@@ -208,6 +213,7 @@ public void startLocalizer(Path nmPrivateContainerTokensPath,
LOG.warn("Exit code from container " + locId + " startLocalizer is : "
+ exitCode, e);
logOutput(shExec.getOutput());
+ xtrace.log("Container finished with exit code "+exitCode);
throw new IOException("Application " + appId + " initialization failed" +
" (exitCode=" + exitCode + ") with output: " + shExec.getOutput(), e);
}
@@ -249,6 +255,7 @@ public int launchContainer(Container container,
container.getLaunchContext().getEnvironment()); // sanitized env
// DEBUG
LOG.info("launchContainer: " + Arrays.toString(commandArray));
+ xtrace.log("Launching container", "args", Arrays.toString(commandArray));
shExec.execute();
if (LOG.isDebugEnabled()) {
logOutput(shExec.getOutput());
@@ -263,6 +270,7 @@ public int launchContainer(Container container,
}
int exitCode = shExec.getExitCode();
LOG.warn("Exit code from container " + containerId + " is : " + exitCode);
+ xtrace.log("Container Finished with exit code "+exitCode);
// 143 (SIGTERM) and 137 (SIGKILL) exit codes means the container was
// terminated/killed forcefully. In all other cases, log the
// container-executor's output
@@ -304,6 +312,7 @@ public boolean signalContainer(String user, String pid, Signal signal)
if (LOG.isDebugEnabled()) {
LOG.debug("signalContainer: " + Arrays.toString(command));
}
+ xtrace.log("signalContainer: Signalling container with signal " + signal.toString(), "args", Arrays.toString(command));
try {
shExec.execute();
} catch (ExitCodeException e) {
@@ -314,6 +323,8 @@ public boolean signalContainer(String user, String pid, Signal signal)
LOG.warn("Error in signalling container " + pid + " with " + signal
+ "; exit = " + ret_code, e);
logOutput(shExec.getOutput());
+ xtrace.log("signalContainer: Problem signalling container, exit code "+ret_code,
+ "Process ID", pid, "Signal", signal);
throw new IOException("Problem signalling container " + pid + " with "
+ signal + "; output: " + shExec.getOutput() + " and exitCode: "
+ ret_code, e);
@@ -343,6 +354,7 @@ public void deleteAsUser(String user, Path dir, Path... baseDirs) {
if (LOG.isDebugEnabled()) {
LOG.debug("deleteAsUser: " + Arrays.toString(commandArray));
}
+ xtrace.log("Delete as user ", "args", Arrays.toString(commandArray));
try {
shExec.execute();
if (LOG.isDebugEnabled()) {
@@ -354,6 +366,7 @@ public void deleteAsUser(String user, Path dir, Path... baseDirs) {
+ " returned with exit code: " + exitCode, e);
LOG.error("Output from LinuxContainerExecutor's deleteAsUser follows:");
logOutput(shExec.getOutput());
+ xtrace.log("deleteAsUser exit code from container is: " + exitCode);
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 8169677bd42a..2b401812a878 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -65,6 +65,9 @@
import com.google.common.annotations.VisibleForTesting;
+import edu.brown.cs.systems.xtrace.XTrace;
+
+
public class NodeStatusUpdaterImpl extends AbstractService implements
NodeStatusUpdater {
@@ -346,13 +349,14 @@ public long getRMIdentifier() {
}
protected void startStatusUpdater() {
-
+
statusUpdaterRunnable = new Runnable() {
@Override
@SuppressWarnings("unchecked")
public void run() {
int lastHeartBeatID = 0;
while (!isStopped) {
+ XTrace.stop();
// Send heartbeat
try {
NodeHeartbeatResponse response = null;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/LocalResourceStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/LocalResourceStatus.java
index d3c5a8f2bccd..4d6205f6ff48 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/LocalResourceStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/LocalResourceStatus.java
@@ -33,4 +33,7 @@ public interface LocalResourceStatus {
public void setLocalPath(URL localPath);
public void setLocalSize(long size);
public void setException(SerializedException exception);
+
+ public void rememberContext();
+ public void joinContext();
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/LocalResourceStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/LocalResourceStatusPBImpl.java
index bc5fcd4f689f..4e8dc107a184 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/LocalResourceStatusPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/LocalResourceStatusPBImpl.java
@@ -33,6 +33,10 @@
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
+import com.google.protobuf.ByteString;
+
+import edu.brown.cs.systems.xtrace.XTrace;
+
public class LocalResourceStatusPBImpl
extends ProtoBase implements LocalResourceStatus {
@@ -221,4 +225,18 @@ private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
return ((SerializedExceptionPBImpl)t).getProto();
}
+ @Override
+ public void rememberContext() {
+ maybeInitBuilder();
+ if (XTrace.active())
+ builder.setXtrace(ByteString.copyFrom(XTrace.bytes()));
+ }
+
+ @Override
+ public void joinContext() {
+ LocalResourceStatusProtoOrBuilder p = viaProto ? proto : builder;
+ if (p.hasXtrace())
+ XTrace.join(p.getXtrace().toByteArray());
+ }
+
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
index 955ccbf19ea1..9667993d6aa0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
@@ -158,6 +158,7 @@ public void stateChanged(Service service) {
@Override
public void handle(AuxServicesEvent event) {
+ event.joinContext();
LOG.info("Got event " + event.getType() + " for appId "
+ event.getApplicationID());
switch (event.getType()) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 26998c938cad..edc5bde31eec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -115,10 +115,13 @@
import com.google.common.annotations.VisibleForTesting;
+import edu.brown.cs.systems.xtrace.XTrace;
+
public class ContainerManagerImpl extends CompositeService implements
ServiceStateChangeListener, ContainerManagementProtocol,
EventHandler {
+ private static final XTrace.Logger xtrace = XTrace.getLogger(ContainerManagerImpl.class);
private static final Log LOG = LogFactory.getLog(ContainerManagerImpl.class);
final Context context;
@@ -454,7 +457,8 @@ private void startContainerInternal(NMTokenIdentifier nmTokenIdentifier,
String user = containerTokenIdentifier.getApplicationSubmitter();
LOG.info("Start request for " + containerIdStr + " by user " + user);
-
+ xtrace.log("Starting container", "User", user, "Container ID", containerIdStr);
+
ContainerLaunchContext launchContext = request.getContainerLaunchContext();
Credentials credentials = parseCredentials(launchContext);
@@ -580,6 +584,8 @@ private void stopContainerInternal(NMTokenIdentifier nmTokenIdentifier,
LOG.info("Stopping container with container Id: " + containerIDStr);
authorizeGetAndStopContainerRequest(containerID, container, true,
nmTokenIdentifier);
+ if (container!=null)
+ container.getContainerId().rememberContext();
dispatcher.getEventHandler().handle(
new ContainerKillEvent(containerID,
@@ -674,6 +680,7 @@ protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
class ContainerEventDispatcher implements EventHandler {
@Override
public void handle(ContainerEvent event) {
+ event.joinContext();
Map containers =
ContainerManagerImpl.this.context.getContainers();
Container c = containers.get(event.getContainerID());
@@ -690,6 +697,7 @@ class ApplicationEventDispatcher implements EventHandler {
@Override
public void handle(ApplicationEvent event) {
+ event.joinContext();
Application app =
ContainerManagerImpl.this.context.getApplications().get(
event.getApplicationID());
@@ -705,6 +713,7 @@ public void handle(ApplicationEvent event) {
@SuppressWarnings("unchecked")
@Override
public void handle(ContainerManagerEvent event) {
+ event.joinContext();
switch (event.getType()) {
case FINISH_APPS:
CMgrCompletedAppsEvent appsFinishedEvent =
@@ -718,8 +727,10 @@ public void handle(ContainerManagerEvent event) {
case FINISH_CONTAINERS:
CMgrCompletedContainersEvent containersFinishedEvent =
(CMgrCompletedContainersEvent) event;
+ XTrace.stop();
for (ContainerId container : containersFinishedEvent
.getContainersToCleanup()) {
+ container.joinContext();
String diagnostic = "";
if (containersFinishedEvent.getReason() ==
CMgrCompletedContainersEvent.Reason.ON_SHUTDOWN) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
index 104896568bfa..2e385434619a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
@@ -18,8 +18,10 @@
package org.apache.hadoop.yarn.server.nodemanager.containermanager.application;
+import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Map;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
@@ -52,6 +54,8 @@
import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory;
+import edu.brown.cs.systems.xtrace.XTrace;
+
/**
* The state machine for the representation of an Application
* within the NodeManager.
@@ -124,7 +128,7 @@ public Map getContainers() {
private static StateMachineFactory stateMachineFactory =
new StateMachineFactory(ApplicationState.NEW)
+ ApplicationEventType, ApplicationEvent>(ApplicationState.NEW, StateMachineFactory.Trace.KEEPALIVE)
// Transitions from NEW state
.addTransition(ApplicationState.NEW, ApplicationState.INITING,
@@ -299,10 +303,16 @@ static class AppInitDoneTransition implements
@Override
public void transition(ApplicationImpl app, ApplicationEvent event) {
// Start all the containers waiting for ApplicationInit
+ edu.brown.cs.systems.xtrace.Context start_context = XTrace.get();
+ Collection end_contexts = new HashSet();
for (Container container : app.containers.values()) {
+ XTrace.set(start_context);
app.dispatcher.getEventHandler().handle(new ContainerInitEvent(
container.getContainerId()));
+ end_contexts.add(XTrace.get());
}
+ for (edu.brown.cs.systems.xtrace.Context ctx : end_contexts)
+ XTrace.join(ctx);
}
}
@@ -411,6 +421,7 @@ public void transition(ApplicationImpl app, ApplicationEvent event) {
@Override
public void handle(ApplicationEvent event) {
+ event.joinContext();
this.writeLock.lock();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index f0cf9db75a9b..82c77d6d5463 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -18,13 +18,13 @@
package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
-import java.io.IOException;
import java.net.URISyntaxException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.concurrent.locks.Lock;
@@ -70,6 +70,9 @@
import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.ConverterUtils;
+import edu.brown.cs.systems.xtrace.Context;
+import edu.brown.cs.systems.xtrace.XTrace;
+
public class ContainerImpl implements Container {
private final Lock readLock;
@@ -99,6 +102,8 @@ public class ContainerImpl implements Container {
new ArrayList