From c52b51dc2ccd0cd657d246532ad312027f21cd2d Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 23 Sep 2013 12:48:07 -0400 Subject: [PATCH 001/112] Applied XTrace patch --- hadoop-common-project/hadoop-common/pom.xml | 5 + .../java/org/apache/hadoop/ipc/Client.java | 47 +++- .../apache/hadoop/ipc/ProtobufRpcEngine.java | 33 ++- .../java/org/apache/hadoop/ipc/Server.java | 43 ++- .../java/org/apache/hadoop/net/NetUtils.java | 17 +- .../org/apache/hadoop/util/ProtoUtil.java | 4 + .../java/org/apache/hadoop/util/Shell.java | 14 + .../src/main/proto/RpcHeader.proto | 2 + .../hadoop/mapred/LocalContainerLauncher.java | 8 + .../mapred/TaskAttemptListenerImpl.java | 11 +- .../org/apache/hadoop/mapred/YarnChild.java | 13 + .../jobhistory/JobHistoryEventHandler.java | 8 + .../hadoop/mapreduce/v2/app/MRAppMaster.java | 30 +- .../v2/app/TaskHeartbeatHandler.java | 9 +- .../hadoop/mapreduce/v2/app/job/Job.java | 2 + .../mapreduce/v2/app/job/impl/JobImpl.java | 86 +++++- .../v2/app/job/impl/TaskAttemptImpl.java | 5 +- .../mapreduce/v2/app/job/impl/TaskImpl.java | 3 +- .../app/launcher/ContainerLauncherImpl.java | 15 +- .../v2/app/local/LocalContainerAllocator.java | 1 + .../mapreduce/v2/app/rm/RMCommunicator.java | 4 + .../v2/app/rm/RMContainerAllocator.java | 84 +++++- .../v2/app/rm/RMContainerRequestor.java | 64 ++++- .../v2/app/speculate/DefaultSpeculator.java | 6 + .../apache/hadoop/mapreduce/v2/app/MRApp.java | 1 + .../hadoop/mapreduce/v2/app/MockJobs.java | 10 + .../v2/app/TestRuntimeEstimators.java | 10 + .../apache/hadoop/mapred/LocalJobRunner.java | 22 +- .../v2/api/records/TaskAttemptId.java | 16 ++ .../apache/hadoop/mapred/CleanupQueue.java | 7 + .../org/apache/hadoop/mapred/IndexRecord.java | 51 +++- .../apache/hadoop/mapred/JobEndNotifier.java | 4 +- .../org/apache/hadoop/mapred/MapTask.java | 139 +++++++++- .../java/org/apache/hadoop/mapred/Merger.java | 4 + .../org/apache/hadoop/mapred/ReduceTask.java | 7 + .../org/apache/hadoop/mapred/SpillRecord.java | 7 +- .../java/org/apache/hadoop/mapred/Task.java | 61 +++++ .../mapred/lib/MultithreadedMapRunner.java | 8 + .../org/apache/hadoop/mapreduce/Cluster.java | 9 +- .../java/org/apache/hadoop/mapreduce/Job.java | 39 +++ .../hadoop/mapreduce/JobSubmissionFiles.java | 2 +- .../apache/hadoop/mapreduce/JobSubmitter.java | 13 +- .../hadoop/mapreduce/lib/chain/Chain.java | 10 + .../lib/map/MultithreadedMapper.java | 7 + .../hadoop/mapreduce/task/reduce/Fetcher.java | 63 ++++- .../task/reduce/InMemoryMapOutput.java | 3 + .../mapreduce/task/reduce/MapOutput.java | 14 + .../task/reduce/MergeManagerImpl.java | 92 ++++++- .../mapreduce/task/reduce/MergeThread.java | 7 +- .../task/reduce/OnDiskMapOutput.java | 3 + .../hadoop/mapreduce/task/reduce/Shuffle.java | 22 +- .../mapreduce/task/reduce/ShuffleHeader.java | 40 +++ .../task/reduce/ShuffleScheduler.java | 2 +- .../task/reduce/ShuffleSchedulerImpl.java | 13 + .../hadoop/mapreduce/v2/hs/CompletedJob.java | 12 + .../mapreduce/v2/hs/HistoryFileManager.java | 6 + .../hadoop/mapreduce/v2/hs/JobHistory.java | 22 ++ .../hadoop/mapreduce/v2/hs/PartialJob.java | 11 + .../hadoop/mapred/ResourceMgrDelegate.java | 5 +- .../org/apache/hadoop/mapred/YARNRunner.java | 1 + .../apache/hadoop/mapred/ShuffleHandler.java | 22 ++ .../org/apache/hadoop/examples/WordCount.java | 12 +- .../hadoop/yarn/api/records/ContainerId.java | 4 + .../yarn/api/records/LocalResource.java | 11 + .../yarn/api/records/ResourceRequest.java | 3 + .../src/main/proto/yarn_protos.proto | 3 + .../yarn/YarnUncaughtExceptionHandler.java | 8 + .../records/impl/pb/ContainerIdPBImpl.java | 28 ++ .../records/impl/pb/LocalResourcePBImpl.java | 24 ++ .../impl/pb/ResourceRequestPBImpl.java | 25 ++ .../hadoop/yarn/event/AbstractEvent.java | 19 ++ .../hadoop/yarn/event/AsyncDispatcher.java | 16 ++ .../org/apache/hadoop/yarn/event/Event.java | 2 + .../impl/pb/RpcClientFactoryPBImpl.java | 1 + .../hadoop/yarn/state/StateMachine.java | 1 + .../yarn/state/StateMachineFactory.java | 257 ++++++++++++++++-- .../apache/hadoop/yarn/util/FSDownload.java | 18 +- .../yarn/server/utils/BuilderUtils.java | 3 + .../nodemanager/DefaultContainerExecutor.java | 12 +- .../server/nodemanager/DeletionService.java | 10 + .../nodemanager/LinuxContainerExecutor.java | 12 + .../nodemanager/NodeStatusUpdaterImpl.java | 8 +- .../protocolrecords/LocalResourceStatus.java | 3 + .../impl/pb/LocalResourceStatusPBImpl.java | 26 ++ .../containermanager/AuxServices.java | 1 + .../ContainerManagerImpl.java | 12 + .../application/ApplicationImpl.java | 13 +- .../container/ContainerImpl.java | 21 +- .../launcher/ContainerLaunch.java | 12 +- .../launcher/ContainersLauncher.java | 5 + .../localizer/ContainerLocalizer.java | 15 +- .../localizer/LocalResourceRequest.java | 8 + .../localizer/LocalResourcesTrackerImpl.java | 3 +- .../localizer/LocalizedResource.java | 5 +- .../ResourceLocalizationService.java | 29 ++ .../logaggregation/LogAggregationService.java | 1 + .../loghandler/NonAggregatingLogHandler.java | 9 + .../monitor/ContainersMonitorImpl.java | 11 +- ...rn_server_nodemanager_service_protos.proto | 1 + .../localizer/MockLocalResourceStatus.java | 5 + .../ApplicationMasterService.java | 3 + .../resourcemanager/ClientRMService.java | 2 + .../resourcemanager/NodesListManager.java | 1 + .../server/resourcemanager/RMAppManager.java | 4 + .../resourcemanager/ResourceManager.java | 9 + .../amlauncher/AMLauncher.java | 21 +- .../amlauncher/ApplicationMasterLauncher.java | 1 + .../resourcemanager/rmapp/RMAppEvent.java | 3 + .../resourcemanager/rmapp/RMAppImpl.java | 11 +- .../rmapp/attempt/RMAppAttemptImpl.java | 7 +- .../rmcontainer/RMContainerImpl.java | 4 +- .../resourcemanager/rmnode/RMNodeImpl.java | 17 +- .../scheduler/capacity/CapacityScheduler.java | 11 + .../scheduler/capacity/LeafQueue.java | 14 +- .../common/fica/FiCaSchedulerApp.java | 6 + .../scheduler/fair/FairScheduler.java | 11 + .../scheduler/fifo/FifoScheduler.java | 23 ++ .../server/resourcemanager/NodeManager.java | 2 +- 118 files changed, 1985 insertions(+), 101 deletions(-) diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml index f264b48adff2..de176400321e 100644 --- a/hadoop-common-project/hadoop-common/pom.xml +++ b/hadoop-common-project/hadoop-common/pom.xml @@ -254,6 +254,11 @@ commons-compress 1.4 + + edu.berkeley.xtrace + xtrace + 2.1-20120824 + diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index fb504dc84bb0..8b9b9e87c7ad 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -37,6 +37,7 @@ import java.net.UnknownHostException; import java.security.PrivilegedExceptionAction; import java.util.Arrays; +import java.util.Collection; import java.util.Hashtable; import java.util.Iterator; import java.util.Map.Entry; @@ -93,6 +94,12 @@ import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ByteString; + +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; +import edu.berkeley.xtrace.XTraceProcess; /** A client for an IPC service. IPC calls take a single {@link Writable} as a * parameter, and return a {@link Writable} as their value. A service runs on @@ -255,6 +262,7 @@ static class Call { IOException error; // exception, null if success final RPC.RpcKind rpcKind; // Rpc EngineKind boolean done; // true when call is done + Collection xtrace; // X-Trace context for the return private Call(RPC.RpcKind rpcKind, Writable param) { this.rpcKind = rpcKind; @@ -631,6 +639,8 @@ private synchronized void setupIOstreams() { if (socket != null || shouldCloseConnection.get()) { return; } + XTraceContext.logEvent(Connection.class, "RPC Client", "Connecting to server"); + Collection start_context = XTraceContext.getThreadContext(); try { if (LOG.isDebugEnabled()) { LOG.debug("Connecting to "+server); @@ -702,9 +712,13 @@ public AuthMethod run() // start the receiver thread after the socket connection has been set // up start(); + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(Connection.class, "RPC Client", "Connected to server"); return; } } catch (Throwable t) { + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(Connection.class, "RPC Client", "Failed to connect to server: "+t.getClass().getName(), "Message", t.getMessage()); if (t instanceof IOException) { markClosed((IOException)t); } else { @@ -1004,8 +1018,17 @@ private void receiveRpcResponse() { if (LOG.isDebugEnabled()) LOG.debug(getName() + " got value #" + callId); - Call call = calls.get(callId); + Call call = calls.get(callId); RpcStatusProto status = header.getStatus(); + if (header.hasXtrace()) { + ByteString xbs = header.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), + 0, xbs.size()); + if (xmd.isValid()) { + call.xtrace = new XTraceMetadataCollection(xmd); + } + } + if (status == RpcStatusProto.SUCCESS) { Writable value = ReflectionUtils.newInstance(valueClass, conf); value.readFields(in); // read value @@ -1316,6 +1339,10 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, ConnectionId remoteId, int serviceClass) throws IOException { final Call call = createCall(rpcKind, rpcRequest); Connection connection = getConnection(remoteId, call, serviceClass); + if (XTraceContext.isValid()) { + XTraceContext.logEvent(RPC.class, "RPC Client", "Sending RPC request", "Call ID", call.id); + call.xtrace = XTraceContext.getThreadContext(); + } try { connection.sendRpcRequest(call); // send the rpc request } catch (RejectedExecutionException e) { @@ -1342,19 +1369,35 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, Thread.currentThread().interrupt(); } + + if (call.xtrace != null) { + XTraceContext.setThreadContext(call.xtrace); + } if (call.error != null) { if (call.error instanceof RemoteException) { call.error.fillInStackTrace(); + if (XTraceContext.isValid()) { + XTraceContext.logEvent(RPC.class, "RPC Client", "RPC response received remote exception", + "Call ID", call.id, "Message", call.error.getMessage()); + } throw call.error; } else { // local exception InetSocketAddress address = connection.getRemoteAddress(); - throw NetUtils.wrapException(address.getHostName(), + IOException e = NetUtils.wrapException(address.getHostName(), address.getPort(), NetUtils.getHostname(), 0, call.error); + if (XTraceContext.isValid()) { + XTraceContext.logEvent(RPC.class, "RPC Client", "Local exception handling RPC response", + "Call ID", call.id, "Message", e.getMessage()); + } + throw e; } } else { + if (XTraceContext.isValid()) { + XTraceContext.logEvent(RPC.class, "RPC Client", "Received RPC response", "Call ID", call.id); + } return call.getRpcResponse(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java index af8e11d78d83..09383cd387c2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java @@ -25,6 +25,7 @@ import java.lang.reflect.Method; import java.lang.reflect.Proxy; import java.net.InetSocketAddress; +import java.util.Collection; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -58,6 +59,10 @@ import com.google.protobuf.ServiceException; import com.google.protobuf.TextFormat; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceProcess; + /** * RPC Engine for for protobuf based RPCs. */ @@ -176,6 +181,11 @@ private RequestHeaderProto constructRpcRequestHeader(Method method) { @Override public Object invoke(Object proxy, Method method, Object[] args) throws ServiceException { + + XTraceContext.logEvent(RPC.class, "ProtobufRpcEngine", "RPC Client invoking remote method "+method.getName(), "Protocol", this.protocolName, "ConnectionID", this.remoteId); + Collection start_context = XTraceContext.getThreadContext(); + try { // xtrace try + long startTime = 0; if (LOG.isDebugEnabled()) { startTime = Time.now(); @@ -241,7 +251,16 @@ public Object invoke(Object proxy, Method method, Object[] args) } catch (Throwable e) { throw new ServiceException(e); } + + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(RPC.class, "ProtobufRpcEngine", "Client invocation of "+method.getName()+" successful"); + return returnMessage; + } catch (ServiceException e) {// xtrace catch + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(RPC.class, "ProtobufRpcEngine", "Remote invocation of "+method.getName()+" failed due to exception: "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } } @Override @@ -563,6 +582,10 @@ public Writable call(RPC.Server server, String protocol, if (server.verbose) LOG.info("Call: protocol=" + protocol + ", method=" + methodName); + XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Invoking method "+methodName, "Protocol", protocol); + Collection start_context = XTraceContext.getThreadContext(); + try { // xtrace try + ProtoClassProtoImpl protocolImpl = getProtocolImpl(server, protoName, clientVersion); BlockingService service = (BlockingService) protocolImpl.protocolImpl; @@ -598,7 +621,15 @@ public Writable call(RPC.Server server, String protocol, } catch (Exception e) { throw e; } - return new RpcResponseWrapper(result); + + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Invocation of "+methodName+" completed, responding to client"); + return new RpcResponseWrapper(result); + } catch (Exception e) { // xtrace catch + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Failed to invoke method "+methodName+": "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index 5e32e7068bbe..c13f85f18cc1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -44,6 +44,7 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -113,6 +114,9 @@ import com.google.protobuf.Message; import com.google.protobuf.Message.Builder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** An abstract IPC service. IPC calls take a single {@link Writable} as a * parameter, and return a {@link Writable} as their value. A service runs on * a port and is defined by a parameter class and a value class. @@ -477,6 +481,7 @@ public static class Call { private ByteBuffer rpcResponse; // the response for this call private final RPC.RpcKind rpcKind; private final byte[] clientId; + private Collection xtrace; // the X-Trace context this was received with public Call(int id, int retryCount, Writable param, Connection connection) { @@ -494,6 +499,12 @@ public Call(int id, int retryCount, Writable param, Connection connection, this.rpcResponse = null; this.rpcKind = kind; this.clientId = clientId; + if (XTraceContext.isValid()) { + XTraceContext.logEvent(RPC.class, "RPC Server", "Received RPC call", "CallID", id); + this.xtrace = XTraceContext.getThreadContext(); + } + XTraceContext.clearThreadContext(); //nothing happens until call gets dequeued + //we use call.xtrace to keep track } @Override @@ -1809,6 +1820,14 @@ private void processOneRpc(byte[] buf) */ private void checkRpcHeaders(RpcRequestHeaderProto header) throws WrappedRpcServerException { + if (header.hasXtrace()) { + ByteString xbs = header.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), + 0, xbs.size()); + if (xmd.isValid()) { + XTraceContext.setThreadContext(xmd); + } + } if (!header.hasRpcOp()) { String err = " IPC Server: No rpc op in rpcRequestHeader"; throw new WrappedRpcServerException( @@ -2012,8 +2031,14 @@ public void run() { ByteArrayOutputStream buf = new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE); while (running) { + XTraceContext.clearThreadContext(); try { final Call call = callQueue.take(); // pop the queue; maybe blocked here + if (call.xtrace != null) { + XTraceContext.setThreadContext(call.xtrace); + XTraceContext.logEvent(RPC.class, "RPC Server", "Processing RPC call", "From", call.connection, + "Protocol", call.connection.protocolName, "rpcKind", call.rpcKind); + } if (LOG.isDebugEnabled()) { LOG.debug(getName() + ": " + call + " for RpcKind " + call.rpcKind); } @@ -2291,6 +2316,18 @@ private void setupResponse(ByteArrayOutputStream responseBuf, headerBuilder.setRetryCount(call.retryCount); headerBuilder.setStatus(status); headerBuilder.setServerIpcVersionNum(CURRENT_VERSION); + if (XTraceContext.isValid()) { + XTraceContext.logEvent(RPC.class, "RPC Server", "Sending " + status + " response", + "Connection", call.connection, "Call ID", call.callId); + headerBuilder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + } + /* X-Trace: we have to send in the response the last event in the server + * before the data is sent, and this is not it, there can be more events + * later, related to enqueuing and sending this call. To log them correctly + * here, we'd have to write the metadata after all these events, maybe + * having the X-Trace metadata as a writable after the response. Alternatively, + * we could use the clock within a span to log these. + */ if (status == RpcStatusProto.SUCCESS) { RpcResponseHeaderProto header = headerBuilder.build(); @@ -2340,6 +2377,10 @@ private void setupResponse(ByteArrayOutputStream responseBuf, wrapWithSasl(responseBuf, call); } call.setResponse(ByteBuffer.wrap(responseBuf.toByteArray())); + + if (XTraceContext.isValid()) { + XTraceContext.clearThreadContext(); //to prevent leaking + } } /** @@ -2579,7 +2620,7 @@ private int channelWrite(WritableByteChannel channel, * This is a wrapper around {@link ReadableByteChannel#read(ByteBuffer)}. * If the amount of data is large, it writes to channel in smaller chunks. * This is to avoid jdk from creating many direct buffers as the size of - * ByteBuffer increases. There should not be any performance degredation. + * ByteBuffer increases. There should not be any performance degradation. * * @see ReadableByteChannel#read(ByteBuffer) */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java index 8595f87b9e8c..7c94beaebe03 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java @@ -58,6 +58,9 @@ import com.google.common.base.Preconditions; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @InterfaceStability.Unstable public class NetUtils { @@ -509,6 +512,10 @@ public static void connect(Socket socket, if (socket == null || endpoint == null || timeout < 0) { throw new IllegalArgumentException("Illegal argument for connect()"); } + + XTraceContext.logEvent(NetUtils.class, "NetUtils", "Connecting to remote...", "Socket", socket.toString(), "Timeout", timeout); + Collection start_context = XTraceContext.getThreadContext(); + try { // xtrace try SocketChannel ch = socket.getChannel(); @@ -546,6 +553,14 @@ public static void connect(Socket socket, "Localhost targeted connection resulted in a loopback. " + "No daemon is listening on the target port."); } + + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(NetUtils.class, "NetUtils", "Connected to remote host"); + } catch (IOException e) { // xtrace catch + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(NetUtils.class, "NetUtils", "Failed to connect to remote host: "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } } /** @@ -904,4 +919,4 @@ public static int getFreeSocketPort() { } return port; } -} +} \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java index 79f8692842dc..67ea33ad2cb1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java @@ -29,6 +29,7 @@ import org.apache.hadoop.security.UserGroupInformation; import com.google.protobuf.ByteString; +import edu.berkeley.xtrace.XTraceContext; public abstract class ProtoUtil { @@ -165,6 +166,9 @@ public static RpcRequestHeaderProto makeRpcRequestHeader(RPC.RpcKind rpcKind, RpcRequestHeaderProto.Builder result = RpcRequestHeaderProto.newBuilder(); result.setRpcKind(convert(rpcKind)).setRpcOp(operation).setCallId(callId) .setRetryCount(retryCount).setClientId(ByteString.copyFrom(uuid)); + if (XTraceContext.isValid()) { + result.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + } return result.build(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java index 49edea0b142d..f185dd0bb5eb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java @@ -32,6 +32,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * A base class for running a Unix command. * @@ -384,6 +387,14 @@ private void runCommand() throws IOException { if (environment != null) { builder.environment().putAll(this.environment); } + // put xtrace context if there is one, merging as appropriate + XTraceMetadata child_process_xtrace = null; + if (XTraceContext.isValid()) { + child_process_xtrace = XTraceContext.startChildProcess(); + builder.environment().put(XTraceContext.XTRACE_CONTEXT_ENV_VARIABLE, XTraceContext.logMerge().toString()); + builder.environment().put(XTraceContext.XTRACE_SUBPROCESS_ENV_VARIABLE, child_process_xtrace.toString()); + } + if (dir != null) { builder.directory(this.dir); } @@ -445,6 +456,9 @@ public void run() { } // wait for the process to finish and check the exit code exitCode = process.waitFor(); + + // join the xtrace process + XTraceContext.joinChildProcess(child_process_xtrace); try { // make sure that the error thread exits errThread.join(); diff --git a/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto b/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto index 673883b23a59..44759d15cdd1 100644 --- a/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto +++ b/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto @@ -67,6 +67,7 @@ message RpcRequestHeaderProto { // the header for the RpcRequest // clientId + callId uniquely identifies a request // retry count, 1 means this is the first retry optional sint32 retryCount = 5 [default = -1]; + optional bytes xtrace = 6; // X-Trace context } @@ -132,6 +133,7 @@ message RpcResponseHeaderProto { optional RpcErrorCodeProto errorDetail = 6; // in case of error optional bytes clientId = 7; // Globally unique client ID optional sint32 retryCount = 8 [default = -1]; + optional bytes xtrace = 9; // X-Trace context } message RpcSaslProto { 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..4ef57d6bcde8 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 @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; +import java.util.Collection; import java.util.HashSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -53,6 +54,9 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * Runs the container task locally in a thread. * Since all (sub)tasks share the same local directory, they must be executed @@ -172,12 +176,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()) { + XTraceContext.clearThreadContext(); 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 +247,8 @@ public void run() { // (i.e., exit clumsily--but can never happen, so no worries!) LOG.fatal("oopsie... this can never happen: " + StringUtils.stringifyException(ioe)); + 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..35121dac4948 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.berkeley.xtrace.XTraceContext; + /** * This class is responsible for talking to the task umblical. * It also converts all the old data structures @@ -434,8 +436,11 @@ 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()); + XTraceContext.logEvent(TaskAttemptListenerImpl.class, "TaskUmbilical getTask", + "Sending task to JVM", "Task ID", task.getTaskID(), "JVM ID", jvmId); jvmTask = new JvmTask(task, false); } } @@ -449,7 +454,11 @@ 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(); + XTraceContext.logEvent(TaskAttemptListenerImpl.class, "TaskUmbilical registerPendingTask", + "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..058b5abe9399 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,6 +62,8 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.log4j.LogManager; +import edu.berkeley.xtrace.XTraceContext; + /** * The main() for MapReduce task processes. */ @@ -72,6 +74,7 @@ class YarnChild { static volatile TaskAttemptID taskid = null; public static void main(String[] args) throws Throwable { + XTraceContext.logEvent(YarnChild.class, "YarnChild", "YarnChild starting"); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); LOG.debug("Child starting"); @@ -133,6 +136,7 @@ public TaskUmbilicalProtocol run() throws Exception { myTask = umbilical.getTask(context); } if (myTask.shouldDie()) { + XTraceContext.logEvent(YarnChild.class, "YarnChild", "Task has been instructed to die."); return; } @@ -159,14 +163,19 @@ public TaskUmbilicalProtocol run() throws Exception { public Object run() throws Exception { // use job-specified working directory FileSystem.get(job).setWorkingDirectory(job.getWorkingDirectory()); + XTraceContext.logEvent(YarnChild.class, "YarnChild", "Running task"); taskFinal.run(job, umbilical); // run the task return null; } }); } catch (FSError e) { + XTraceContext.logEvent(YarnChild.class, "YarnChild FSError", "FSError from child: "+e.getClass().getName(), + "Message", e.getMessage()); LOG.fatal("FSError from child", e); umbilical.fsError(taskid, e.getMessage()); } catch (Exception exception) { + XTraceContext.logEvent(YarnChild.class, "YarnChild Exception", "Exception running child: "+exception.getClass().getName(), + "Message", exception.getMessage()); LOG.warn("Exception running child : " + StringUtils.stringifyException(exception)); try { @@ -193,6 +202,8 @@ public Object run() throws Exception { umbilical.fatalError(taskid, StringUtils.stringifyException(exception)); } } catch (Throwable throwable) { + XTraceContext.logEvent(YarnChild.class, "YarnChild Error", "Error running child: "+throwable.getClass().getName(), + "Message", throwable.getMessage()); LOG.fatal("Error running child : " + StringUtils.stringifyException(throwable)); if (taskid != null) { @@ -203,6 +214,8 @@ public Object run() throws Exception { umbilical.fatalError(taskid, cause); } } finally { + XTraceContext.logEvent(YarnChild.class,"YarnChild", "YarnChild exiting"); + 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..d910484d2c44 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.mapreduce.jobhistory; import java.io.IOException; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; @@ -56,6 +57,9 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * 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 +263,15 @@ public void run() { eventCounter++; } + XTraceContext.clearThreadContext(); 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 +487,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..22c877572e63 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 @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; +import java.util.Collection; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -134,6 +135,8 @@ import org.apache.hadoop.yarn.util.SystemClock; import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; /** * The Map-Reduce Application Master. @@ -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 */ + XTraceContext.logEvent(MRAppMaster.class, "MRAppMaster", "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,7 @@ public void cleanupStagingDir() throws IOException { * Exit call. Just in a function call to enable testing. */ protected void sysexit() { + XTraceContext.joinParentProcess(); System.exit(0); } @@ -514,6 +523,7 @@ public void shutDownJob() { // note in a workflow scenario, this may lead to creation of a new // job (FIXME?) // Send job-end notification + XTraceContext.logEvent(JobFinishEvent.class, "JobFinish", "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 +566,20 @@ public void shutDownJob() { //Bring the process down by force. //Not needed after HADOOP-7140 LOG.info("Exiting MR AppMaster..GoodBye!"); + XTraceContext.logEvent(MRAppMaster.class, "MRAppMaster", "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 +810,7 @@ protected void serviceStop() throws Exception { @Override public void handle(ContainerAllocatorEvent event) { + event.joinContext(); this.containerAllocator.handle(event); } @@ -849,6 +862,7 @@ protected void serviceStart() throws Exception { @Override public void handle(ContainerLauncherEvent event) { + event.joinContext(); this.containerLauncher.handle(event); } @@ -1181,6 +1195,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 +1204,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 +1216,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 +1233,7 @@ public SpeculatorEventDispatcher(Configuration config) { } @Override public void handle(SpeculatorEvent event) { + event.joinContext(); if (disabled) { return; } @@ -1274,6 +1292,7 @@ private static void validateInputParam(String value, String param) public static void main(String[] args) { try { + XTraceContext.startTrace("MRAppMaster", "Application Master Launching"); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); String containerIdStr = System.getenv(Environment.CONTAINER_ID.name()); @@ -1323,6 +1342,8 @@ public static void main(String[] args) { initAndStartAppMaster(appMaster, conf, jobUserName); } catch (Throwable t) { LOG.fatal("Error starting MRAppMaster", t); + XTraceContext.logEvent(MRAppMaster.class, "MRAppMaster", "Error starting MRAppMaster: "+t.getClass().getName(), "Message", t.getMessage()); + XTraceContext.joinParentProcess(); System.exit(1); } } @@ -1331,12 +1352,17 @@ public static void main(String[] args) { // close of the JVM. static class MRAppMasterShutdownHook implements Runnable { MRAppMaster appMaster; + private Collection xtrace_context; MRAppMasterShutdownHook(MRAppMaster appMaster) { this.appMaster = appMaster; + xtrace_context = XTraceContext.getThreadContext(); } public void run() { + XTraceContext.setThreadContext(xtrace_context); LOG.info("MRAppMaster received a signal. Signaling RMCommunicator and " + "JobHistoryEventHandler."); + XTraceContext.logEvent(MRAppMasterShutdownHook.class, "MRAppMasterShutdownHook", + "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,8 @@ public void run() { } appMaster.notifyIsLastAMRetry(appMaster.isLastAMRetry); appMaster.stop(); + XTraceContext.logEvent(MRAppMaster.class,"MRAppMaster", "MRAppMaster stopping"); + XTraceContext.clearThreadContext(); } } 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..b645342722ca 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.v2.app; +import java.util.Collection; import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -35,6 +36,9 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.util.Clock; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * This class keeps track of tasks that have already been launched. It @@ -115,11 +119,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 +150,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)); + XTraceContext.clearThreadContext(); } } 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..bed6447f6a41 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 @@ -125,6 +125,10 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.Clock; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + /** Implementation of Job interface. Maintains the state machines of Job. * The read and write calls use ReadWriteLock for concurrency. */ @@ -233,7 +237,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 +599,14 @@ JobEventType.JOB_KILL, new KillTasksTransition()) private boolean isUber = false; private Credentials jobCredentials; + private Collection completedTaskContexts = new XTraceMetadataCollection(); + private Collection succeededMapTaskContexts = new XTraceMetadataCollection(); + private Collection succeededReduceTaskContexts = new XTraceMetadataCollection(); + private Collection failedMapTaskContexts = new XTraceMetadataCollection(); + private Collection failedReduceTaskContexts = new XTraceMetadataCollection(); + private Collection killedMapTaskContexts = new XTraceMetadataCollection(); + private Collection killedReduceTaskContexts = new XTraceMetadataCollection(); + private Token jobToken; private JobTokenSecretManager jobTokenSecretManager; @@ -692,6 +704,20 @@ public int getCompletedMaps() { readLock.unlock(); } } + + @Override + public int getAndJoinCompletedMaps() { + readLock.lock(); + try { + XTraceContext.joinContext(succeededMapTaskContexts); + XTraceContext.joinContext(failedMapTaskContexts); + XTraceContext.joinContext(killedMapTaskContexts); + return succeededMapTaskCount + failedMapTaskCount + killedMapTaskCount; + } finally { + readLock.unlock(); + } + } + @Override public int getCompletedReduces() { @@ -703,6 +729,20 @@ public int getCompletedReduces() { readLock.unlock(); } } + + @Override + public int getAndJoinCompletedReduces() { + readLock.lock(); + try { + XTraceContext.joinContext(succeededReduceTaskContexts); + XTraceContext.joinContext(failedReduceTaskContexts); + XTraceContext.joinContext(killedReduceTaskContexts); + return succeededReduceTaskCount + failedReduceTaskCount + + killedReduceTaskCount; + } finally { + readLock.unlock(); + } + } @Override public boolean isUber() { @@ -790,6 +830,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 +952,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 +1011,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 +1063,11 @@ protected FileSystem getFileSystem(Configuration conf) throws IOException { protected JobStateInternal checkReadyForCommit() { JobStateInternal currentState = getInternalState(); + joinStateMachineXTraceContext(); if (completedTaskCount == tasks.size() && currentState == JobStateInternal.RUNNING) { + XTraceContext.logEvent(JobImpl.class, "JobImpl checkReadyForCommit", "Job ready for commit.", + "Completed Tasks", tasks.size(), "Current State", currentState); eventHandler.handle(new CommitterJobCommitEvent(jobId, getJobContext())); return JobStateInternal.COMMITTING; } @@ -1025,6 +1079,7 @@ JobStateInternal finished(JobStateInternal finalState) { if (getInternalState() == JobStateInternal.RUNNING) { metrics.endRunningJob(this); } + joinStateMachineXTraceContext(); if (finishTime == 0) setFinishTime(); eventHandler.handle(new JobFinishEvent(jobId)); @@ -1368,6 +1423,7 @@ public JobStateInternal transition(JobImpl job, JobEvent event) { // create the Tasks but don't start them yet createMapTasks(job, inputLength, taskSplitMetaInfo); createReduceTasks(job); + XTraceContext.logEvent(JobImpl.class, "JobImpl", "Created map and reduce tasks"); job.metrics.endPreparingJob(job); return JobStateInternal.INITED; @@ -1424,7 +1480,12 @@ protected void setup(JobImpl job) throws IOException { private void createMapTasks(JobImpl job, long inputLength, TaskSplitMetaInfo[] splits) { + + XTraceContext.logEvent(JobImpl.class, "JobImpl", "Creating Map Tasks", "Input Length", inputLength, + "Num Splits", splits.length); + Collection start_context = XTraceContext.getThreadContext(); for (int i=0; i < job.numMapTasks; ++i) { + XTraceContext.setThreadContext(start_context); TaskImpl task = new MapTaskImpl(job.jobId, i, job.eventHandler, @@ -1442,7 +1503,10 @@ private void createMapTasks(JobImpl job, long inputLength, } private void createReduceTasks(JobImpl job) { + XTraceContext.logEvent(JobImpl.class, "JobImpl", "Creating Reduce Tasks", "Num Reduces", job.numReduceTasks); + Collection start_context = XTraceContext.getThreadContext(); for (int i = 0; i < job.numReduceTasks; i++) { + XTraceContext.setThreadContext(start_context); TaskImpl task = new ReduceTaskImpl(job.jobId, i, job.eventHandler, @@ -1543,6 +1607,8 @@ public void transition(JobImpl job, JobEvent event) { private void unsuccessfulFinish(JobStateInternal finalState) { if (finishTime == 0) setFinishTime(); cleanupProgress = 1.0f; + XTraceContext.joinContext(succeededMapTaskContexts); + XTraceContext.joinContext(succeededReduceTaskContexts); JobUnsuccessfulCompletionEvent unsuccessfulJobEvent = new JobUnsuccessfulCompletionEvent(oldJobId, finishTime, @@ -1571,6 +1637,10 @@ private static JobFinishedEvent createJobFinishedEvent(JobImpl job) { job.mayBeConstructFinalFullCounters(); + XTraceContext.joinContext(job.succeededMapTaskContexts); + XTraceContext.joinContext(job.succeededReduceTaskContexts); + XTraceContext.joinContext(job.failedMapTaskContexts); + XTraceContext.joinContext(job.failedReduceTaskContexts); JobFinishedEvent jfe = new JobFinishedEvent( job.oldJobId, job.finishTime, job.succeededMapTaskCount, job.succeededReduceTaskCount, @@ -1807,6 +1877,12 @@ protected JobStateInternal checkJobAfterTaskCompletion(JobImpl job) { String diagnosticMsg = "Job failed as tasks failed. " + "failedMaps:" + job.failedMapTaskCount + " failedReduces:" + job.failedReduceTaskCount; + + XTraceContext.joinContext(job.failedMapTaskContexts); + XTraceContext.joinContext(job.failedReduceTaskContexts); + XTraceContext.logEvent(JobImpl.class, "JobImpl checkJobAfterTaskCompletion", "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 +1896,10 @@ protected JobStateInternal checkJobAfterTaskCompletion(JobImpl job) { private void taskSucceeded(JobImpl job, Task task) { if (task.getType() == TaskType.MAP) { + job.succeededMapTaskContexts = XTraceContext.getThreadContext(job.succeededMapTaskContexts); job.succeededMapTaskCount++; } else { + job.succeededReduceTaskContexts = XTraceContext.getThreadContext(job.succeededReduceTaskContexts); job.succeededReduceTaskCount++; } job.metrics.completedTask(task); @@ -1829,8 +1907,10 @@ private void taskSucceeded(JobImpl job, Task task) { private void taskFailed(JobImpl job, Task task) { if (task.getType() == TaskType.MAP) { + job.failedMapTaskContexts = XTraceContext.getThreadContext(job.failedMapTaskContexts); job.failedMapTaskCount++; } else if (task.getType() == TaskType.REDUCE) { + job.failedReduceTaskContexts = XTraceContext.getThreadContext(job.failedReduceTaskContexts); job.failedReduceTaskCount++; } job.addDiagnostic("Task failed " + task.getID()); @@ -1839,8 +1919,10 @@ private void taskFailed(JobImpl job, Task task) { private void taskKilled(JobImpl job, Task task) { if (task.getType() == TaskType.MAP) { + job.killedMapTaskContexts = XTraceContext.getThreadContext(job.killedMapTaskContexts); job.killedMapTaskCount++; } else if (task.getType() == TaskType.REDUCE) { + job.killedReduceTaskContexts = XTraceContext.getThreadContext(job.killedReduceTaskContexts); job.killedReduceTaskCount++; } job.metrics.killedTask(task); @@ -1902,6 +1984,8 @@ private static class MapTaskRescheduledTransition implements @Override public void transition(JobImpl job, JobEvent event) { //succeeded map task is restarted back + job.succeededMapTaskContexts = XTraceContext.getThreadContext(job.succeededMapTaskContexts); + job.completedTaskContexts = XTraceContext.getThreadContext(job.completedTaskContexts); 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..9a44e7b16e18 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 @@ -21,6 +21,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.security.PrivilegedAction; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -58,6 +60,9 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * This class is responsible for launching of containers. */ @@ -124,6 +129,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 +196,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 +273,7 @@ public void run() { Set allNodes = new HashSet(); while (!stopped.get() && !Thread.currentThread().isInterrupted()) { + XTraceContext.clearThreadContext(); try { event = eventQueue.take(); } catch (InterruptedException e) { @@ -276,7 +283,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 +361,10 @@ class EventProcessor implements Runnable { @Override public void run() { + XTraceContext.clearThreadContext(); + event.joinContext(); LOG.info("Processing the event " + event.toString()); + XTraceContext.logEvent(ContainerLauncherImpl.class, "ContainerLauncherImpl", "Processing the event " + event.toString()); // Load ContainerManager tokens before creating a connection. // TODO: Do it only once per NodeManager. @@ -374,6 +384,7 @@ public void run() { break; } removeContainerIfDone(containerID); + XTraceContext.clearThreadContext(); } } 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..c0fc462bedd0 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 @@ -21,6 +21,7 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; @@ -54,6 +55,8 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; /** * Registers/unregisters to RM and sends heartbeats to RM. */ @@ -247,6 +250,7 @@ public void run() { } return; } + XTraceContext.clearThreadContext(); } } }); 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..e7cb61787c29 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.mapreduce.v2.app.rm; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -74,6 +75,9 @@ import org.apache.hadoop.yarn.util.RackResolver; import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; /** * Allocates the container from the ResourceManager scheduler. @@ -187,6 +191,7 @@ public void run() { ContainerAllocatorEvent event; while (!stopped.get() && !Thread.currentThread().isInterrupted()) { + XTraceContext.clearThreadContext(); 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(); + Collection start_context = XTraceContext.getThreadContext(); + + 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; } + + XTraceContext.setThreadContext(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) { + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "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 + XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", "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 + XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", "Scheduling Fail-Fast Reduce Container Request"); + reqEvent.rememberContext(); pendingReduces.addFirst(new ContainerRequest(reqEvent, PRIORITY_REDUCE)); } else { + XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", "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) { + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "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) { + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "Processing CONTAINER_FAILED Event"); ContainerFailedEvent fEv = (ContainerFailedEvent) event; String host = getHost(fEv.getContMgrAddress()); containerFailedOnHost(host); @@ -515,18 +535,31 @@ public void scheduleReduces( @Private public void scheduleAllReduces() { + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator scheduleAllReduces", "Scheduling all reduces"); + Collection start_context = XTraceContext.getThreadContext(); for (ContainerRequest req : pendingReduces) { + req.joinContext(); + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator scheduleAllReduces", "Scheduling reduce"); + req.rememberContext(); scheduledRequests.addReduce(req); + XTraceContext.setThreadContext(start_context); } pendingReduces.clear(); } @Private public void rampUpReduces(int rampUp) { + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator rampUpReduces", "Ramping up reduces", + "Ramp Up", rampUp); //more reduce to be scheduled + Collection start_context = XTraceContext.getThreadContext(); for (int i = 0; i < rampUp; i++) { ContainerRequest request = pendingReduces.removeFirst(); + request.joinContext(); + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator rampUpReduces", "Request ramping up"); + request.rememberContext(); scheduledRequests.addReduce(request); + XTraceContext.setThreadContext(start_context); } } @@ -737,6 +770,8 @@ ContainerRequest removeReduce() { } void addMap(ContainerRequestEvent event) { + event.joinContext(); + ContainerRequest request = null; if (event.getEarlierAttemptFailed()) { @@ -783,8 +818,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 +869,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 +909,8 @@ else if (PRIORITY_REDUCE.equals(priority)) { it.remove(); continue; } + + XTraceContext.clearThreadContext(); } assignContainers(allocatedContainers); @@ -879,15 +919,22 @@ 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); + XTraceContext.clearThreadContext(); } + + XTraceContext.clearThreadContext(); } @SuppressWarnings("unchecked") private void containerAssigned(Container allocated, ContainerRequest assigned) { + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", + "Container Assigned", "Container ID", allocated.getId().toString(), "Assigned ID", assigned.attemptID); + // Update resource requests decContainerReq(assigned); @@ -916,11 +963,17 @@ 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(); + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "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(); + XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "Assigned container to reduce", + "Container ID", allocated.getId(), "Assigned ID", assigned.attemptID); } return assigned; @@ -929,6 +982,7 @@ private ContainerRequest assignWithoutLocality(Container allocated) { private void assignContainers(List allocatedContainers) { Iterator it = allocatedContainers.iterator(); while (it.hasNext()) { + XTraceContext.clearThreadContext(); Container allocated = it.next(); ContainerRequest assigned = assignWithoutLocality(allocated); if (assigned != null) { @@ -936,8 +990,10 @@ private void assignContainers(List allocatedContainers) { it.remove(); } } + XTraceContext.clearThreadContext(); assignMapsWithLocality(allocatedContainers); + XTraceContext.clearThreadContext(); } private ContainerRequest getContainerReqToReplace(Container allocated) { @@ -986,6 +1042,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 +1060,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 +1071,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(); + XTraceContext.clearThreadContext(); + 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 +1088,9 @@ private void assignMapsWithLocality(List allocatedContainers) { TaskAttemptId tId = list.removeFirst(); if (maps.containsKey(tId)) { ContainerRequest assigned = maps.remove(tId); + assigned.joinContext(); + XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", + "Assigned container based on host match", "Host", host); containerAssigned(allocated, assigned); it.remove(); JobCounterUpdateEvent jce = @@ -1041,11 +1105,14 @@ private void assignMapsWithLocality(List allocatedContainers) { } } } + XTraceContext.clearThreadContext(); // try to match all rack local it = allocatedContainers.iterator(); while(it.hasNext() && maps.size() > 0){ + XTraceContext.clearThreadContext(); 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 +1124,9 @@ private void assignMapsWithLocality(List allocatedContainers) { TaskAttemptId tId = list.removeFirst(); if (maps.containsKey(tId)) { ContainerRequest assigned = maps.remove(tId); + assigned.joinContext(); + XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", + "Assigned container based on rack match", "Rack", rack); containerAssigned(allocated, assigned); it.remove(); JobCounterUpdateEvent jce = @@ -1069,17 +1139,23 @@ private void assignMapsWithLocality(List allocatedContainers) { } break; } + XTraceContext.clearThreadContext(); } } + XTraceContext.clearThreadContext(); // assign remaining it = allocatedContainers.iterator(); while(it.hasNext() && maps.size() > 0){ + XTraceContext.clearThreadContext(); 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(); + XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", "Assigned container based on * match"); containerAssigned(allocated, assigned); it.remove(); JobCounterUpdateEvent jce = @@ -1089,7 +1165,9 @@ private void assignMapsWithLocality(List allocatedContainers) { if (LOG.isDebugEnabled()) { LOG.debug("Assigned based on * match"); } + XTraceContext.clearThreadContext(); } + XTraceContext.clearThreadContext(); } } 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..cec9d3c930a6 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 @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -38,7 +39,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,6 +50,8 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; /** * Keeps the data structures to send container requests to RM. @@ -98,10 +103,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 +128,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 +165,24 @@ protected void serviceInit(Configuration conf) throws Exception { } protected AllocateResponse makeRemoteRequest() throws IOException { + + for (ResourceRequest r : ask) { + XTraceContext.clearThreadContext(); + r.joinContext(); + XTraceContext.logEvent(RMContainerRequestor.class, "ContainerRequestor", "Requesting container from RM"); + r.rememberContext(); + } + + for (ContainerId i : release) { + XTraceContext.clearThreadContext(); + i.joinContext(); + XTraceContext.logEvent(RMContainerRequestor.class, "ContainerRequestor", "Requesting RM release container", + "Container ID", i); + i.rememberContext(); + } + XTraceContext.clearThreadContext(); + + AllocateRequest allocateRequest = AllocateRequest.newInstance(lastResponseID, super.getApplicationProgress(), new ArrayList(ask), @@ -169,6 +207,22 @@ protected AllocateResponse makeRemoteRequest() throws IOException { + " resourcelimit=" + availableResources + " knownNMs=" + clusterNmCount); } + + for (Container x : allocateResponse.getAllocatedContainers()) { + XTraceContext.clearThreadContext(); + x.getId().joinContext(); + XTraceContext.logEvent(RMContainerRequestor.class, "ContainerRequestor", "Container allocated by RM"); + x.getId().rememberContext(); + } + + for (ContainerStatus x : allocateResponse.getCompletedContainersStatuses()) { + XTraceContext.clearThreadContext(); + x.getContainerId().joinContext(); + XTraceContext.logEvent(RMContainerRequestor.class, "ContainerRequestor", "RM acknowledged completed container"); + x.getContainerId().rememberContext(); + + } + XTraceContext.clearThreadContext(); ask.clear(); release.clear(); @@ -325,7 +379,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 +429,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 +458,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..2f72d923ad6f 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 @@ -20,6 +20,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.util.Collection; import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -52,6 +53,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.util.Clock; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; public class DefaultSpeculator extends AbstractService implements Speculator { @@ -167,10 +170,12 @@ public DefaultSpeculator(Configuration conf, AppContext context, Clock clock) { @Override protected void serviceStart() throws Exception { + final Collection xtrace_context = XTraceContext.getThreadContext(); Runnable speculationBackgroundCore = new Runnable() { @Override public void run() { + XTraceContext.joinContext(xtrace_context); while (!stopped && !Thread.currentThread().isInterrupted()) { long backgroundRunStartTime = clock.getTime(); try { @@ -428,6 +433,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..ea880f923d6a 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 @@ -25,6 +25,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -68,6 +69,9 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** Implements MapReduce locally, in-process, for debugging. */ @InterfaceAudience.Private @InterfaceStability.Unstable @@ -128,6 +132,7 @@ private class Job extends Thread implements TaskUmbilicalProtocol { boolean killed = false; private LocalDistributedCacheManager localDistributedCacheManager; + private Collection xtrace_context; public long getProtocolVersion(String protocol, long clientVersion) { return TaskUmbilicalProtocol.versionID; @@ -178,6 +183,8 @@ public Job(JobID jobid, String jobSubmitDir) throws IOException { profile.getURL().toString()); jobs.put(id, this); + + this.xtrace_context = XTraceContext.getThreadContext(); this.start(); } @@ -197,6 +204,7 @@ protected class MapTaskRunnable implements Runnable { private final Map mapOutputFiles; public volatile Throwable storedException; + private Collection xtrace_context; public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId, Map mapOutputFiles) { @@ -205,9 +213,11 @@ public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId, this.mapOutputFiles = mapOutputFiles; this.jobId = jobId; this.localConf = new JobConf(job); + this.xtrace_context = XTraceContext.getThreadContext(); } public void run() { + XTraceContext.setThreadContext(xtrace_context); try { TaskAttemptID mapId = new TaskAttemptID(new TaskID( jobId, TaskType.MAP, taskId), 0); @@ -240,6 +250,8 @@ public void run() { } catch (Throwable e) { this.storedException = e; } + this.xtrace_context = XTraceContext.getThreadContext(); + XTraceContext.clearThreadContext(); } } @@ -342,6 +354,8 @@ protected ExecutorService createMapExecutor(int numMapTasks) { @Override public void run() { + XTraceContext.setThreadContext(xtrace_context); + JobID jobId = profile.getJobID(); JobContext jContext = new JobContextImpl(job, jobId); @@ -391,7 +405,13 @@ public void run() { mapService.shutdownNow(); throw ie; } - + + XTraceContext.clearThreadContext(); + for (MapTaskRunnable r : taskRunnables) { + XTraceContext.joinContext(r.xtrace_context); + } + + XTraceContext.logEvent(Job.class, "Job", "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..7e45f21e7e1b 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,11 @@ package org.apache.hadoop.mapreduce.v2.api.records; +import java.util.Collection; + +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** *

* TaskAttemptId represents the unique identifier for a task @@ -32,6 +37,17 @@ *

*/ public abstract class TaskAttemptId implements Comparable { + + private Collection xtrace_context; + + public void rememberContext() { + this.xtrace_context = XTraceContext.getThreadContext(); + } + + public void joinContext() { + XTraceContext.joinContext(this.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..66eed58a7132 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.mapred; import java.io.IOException; +import java.util.Collection; import java.util.concurrent.LinkedBlockingQueue; import org.apache.commons.logging.Log; @@ -27,6 +28,9 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + class CleanupQueue { public static final Log LOG = @@ -105,10 +109,12 @@ private static class PathCleanupThread extends Thread { // cleanup queue which deletes files/directories of the paths queued up. private LinkedBlockingQueue queue = new LinkedBlockingQueue(); + private Collection xtrace_context; public PathCleanupThread() { setName("Directory/File cleanup thread"); setDaemon(true); + xtrace_context = XTraceContext.getThreadContext(); start(); } @@ -121,6 +127,7 @@ void addToQueue(PathDeletionContext[] contexts) { } public void run() { + XTraceContext.joinContext(xtrace_context); if (LOG.isDebugEnabled()) { LOG.debug(getName() + " started."); } 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..2028ec2f7823 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 @@ -17,21 +17,70 @@ */ package org.apache.hadoop.mapred; +import java.nio.ByteBuffer; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.TaskID; + @InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceStability.Unstable public class IndexRecord { public long startOffset; public long rawLength; public long partLength; + public XTraceMetadata m; 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 && xtrace_opid!=0) { + byte[] taskid = ByteBuffer.allocate(8).putLong(xtrace_taskid).array(); + byte[] opid = ByteBuffer.allocate(8).putLong(xtrace_opid).array(); + m = new XTraceMetadata(new TaskID(taskid, 8), opid); + } + } + + public long getXTraceTaskID() { + if (m!=null) { + try { + return ByteBuffer.wrap(m.getTaskId().get()).getLong(); + } catch (Exception e) { + } + } + return 0L; + } + + public long getXTraceOpID() { + if (m!=null) { + try { + return ByteBuffer.wrap(m.getOpId()).getLong(); + } catch (Exception e) { + } + } + return 0L; + } + + public void rememberContext() { + m = XTraceContext.logMerge(); + } + + public void clearContext() { + m = null; + } + + public void joinContext() { + XTraceContext.joinContext(m); + } + + public boolean hasContext() { + return m!=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..92952763e067 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,7 +28,10 @@ import java.nio.ByteOrder; import java.nio.IntBuffer; import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -73,6 +76,10 @@ import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.util.StringUtils; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + /** A Map task. */ @InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceStability.Unstable @@ -80,7 +87,7 @@ 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; @@ -258,6 +265,10 @@ 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 +461,7 @@ void runOldMapper(final JobConf job, closeQuietly(in); closeQuietly(collector); } + XTraceContext.logEvent(MapTask.class, "OldMapper", "Sort phase end"); } /** @@ -730,6 +748,7 @@ void runNewMapper(final JobConf job, split = getSplitDetails(new Path(splitIndex.getSplitLocation()), splitIndex.getStartOffset()); LOG.info("Processing split: " + split); + XTraceContext.logEvent(MapTask.class, "NewMapper", "Processing split", "Split", split); org.apache.hadoop.mapreduce.RecordReader input = new NewTrackingRecordReader @@ -760,14 +779,18 @@ void runNewMapper(final JobConf job, try { input.initialize(split, mapperContext); + XTraceContext.logEvent(MapTask.class, "NewMapper", "Map start"); mapper.run(mapperContext); mapPhase.complete(); + XTraceContext.logEvent(MapTask.class, "NewMapper", "Map end"); + XTraceContext.logEvent(MapTask.class, "NewMapper", "Sort phase start"); setPhase(TaskStatus.Phase.SORT); statusUpdate(umbilical); input.close(); input = null; output.close(mapperContext); output = null; + XTraceContext.logEvent(MapTask.class, "NewMapper", "Sort phase end"); } finally { closeQuietly(input); closeQuietly(output, mapperContext); @@ -1087,6 +1110,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 + XTraceContext.logEvent(MapTask.class, "MapTask startspill async", "Triggering asynchronous spill"); startSpill(); final int avgRec = (int) (mapOutputByteCounter.getCounter() / @@ -1158,6 +1182,8 @@ 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()); + XTraceContext.logEvent(MapOutputBuffer.class, "MapTask Output Buffer Collect", + "Record too large for in-memory buffer", "Message", e.getMessage()); spillSingleRecord(key, value, partition); mapOutputRecordCounter.increment(1); return; @@ -1390,6 +1416,7 @@ public void write(byte b[], int off, int len) } // we have records we can spill; only spill if blocked if (kvindex != kvend) { + XTraceContext.logEvent(MapTask.class, "MapTask startspill sync", "Triggering synchronous spill"); startSpill(); // Blocked on this write, waiting for the spill just // initiated to finish. Instead of repositioning the marker @@ -1417,6 +1444,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 +1471,8 @@ public void write(byte b[], int off, int len) public void flush() throws IOException, ClassNotFoundException, InterruptedException { LOG.info("Starting flush of map output"); + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer flush", "Starting flush of map output"); + Collection start_context = XTraceContext.getThreadContext(); spillLock.lock(); try { while (spillInProgress) { @@ -1500,16 +1530,40 @@ public void close() { } protected class SpillThread extends Thread { + private Collection xtrace_spillstart_context; + private Collection xtrace_spilldone_context; + + public void rememberSpillStartContext() { + xtrace_spillstart_context = XTraceContext.getThreadContext(); + } + + public void rememberSpillDoneContext() { + xtrace_spilldone_context = XTraceContext.getThreadContext(); + } + + public void joinSpillDoneContext() { + XTraceContext.joinContext(xtrace_spilldone_context); + xtrace_spilldone_context = null; + } + + public void joinSpillStartContext() { + XTraceContext.joinContext(xtrace_spillstart_context); + xtrace_spillstart_context = null; + } + @Override public void run() { spillLock.lock(); spillThreadRunning = true; try { while (true) { + XTraceContext.clearThreadContext(); spillDone.signal(); while (!spillInProgress) { spillReady.await(); } + joinSpillStartContext(); + XTraceContext.logEvent(SpillThread.class, "SpillThread", "Spill Thread notified"); try { spillLock.unlock(); sortAndSpill(); @@ -1524,6 +1578,7 @@ public void run() { bufstart = bufend; spillInProgress = false; } + rememberSpillDoneContext(); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -1560,11 +1615,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 + XTraceContext.logEvent(MapTask.class, "MapOutputBuffer sortAndSpill", "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 +1643,20 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, (kvstart >= kvend ? kvstart : kvmeta.capacity() + kvstart) / NMETA; + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer sort", "Sorting buffer contents"); sorter.sort(MapOutputBuffer.this, mstart, mend, reporter); int spindex = mstart; final IndexRecord rec = new IndexRecord(); final InMemValBytes value = new InMemValBytes(); + + + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", + "Spilling partitions to disk", "NumPartitions", partitions, "File", filename); + Collection start_context = XTraceContext.getThreadContext(); + Collection end_context = new XTraceMetadataCollection(); for (int i = 0; i < partitions; ++i) { + XTraceContext.setThreadContext(start_context); + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", "Spilling partition", "Partition Number", i); IFile.Writer writer = null; try { long segmentStart = out.getPos(); @@ -1597,6 +1665,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 +1675,15 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, getVBytesForOffset(kvoff, value); writer.append(key, value); ++spindex; + ++spillcount; + } + if (spillcount > 0) { + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", + "Spilled records directly without combine", "Partition", i, "SpillCount", spillcount); + rec.rememberContext(); + } else { + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", "No records to spill", "Partition", i); + rec.clearContext(); } } else { int spstart = spindex; @@ -1621,6 +1699,12 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, RawKeyValueIterator kvIter = new MRResultIterator(spstart, spindex); combinerRunner.combine(kvIter, combineCollector); + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", + "Spilled records with combine", "Partition", i, "SpillCount", (spindex-spstart)); + rec.rememberContext(); + } else { + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", "No records to spill", "Partition", i); + rec.clearContext(); } } @@ -1636,8 +1720,10 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, writer = null; } finally { if (null != writer) writer.close(); + end_context = XTraceContext.getThreadContext(end_context); } } + XTraceContext.joinContext(end_context); if (totalIndexCacheMemory >= indexCacheMemoryLimit) { // create spill index file @@ -1651,6 +1737,7 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH; } LOG.info("Finished spill " + numSpills); + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer sortAndSpill", "Finished spill", "Spill Number", numSpills); ++numSpills; } finally { if (out != null) out.close(); @@ -1689,6 +1776,9 @@ 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); + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spillSingleRecord", + "Spilled single record", "Partition", partition); + rec.rememberContext(); } writer.close(); @@ -1794,6 +1884,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 +1895,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 + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer mergeParts", "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); + Collection start_context = XTraceContext.getThreadContext(); + for (int i = 0; i < partitions; i++) { + IndexRecord rec = spillRec.getIndex(i); + if (rec.hasContext()) { + rec.joinContext(); + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer rename", "Spill file containing partition was renamed"); + rec.rememberContext(); + spillRec.putIndex(rec, i); + XTraceContext.setThreadContext(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 +1970,16 @@ private void mergeParts() throws IOException, InterruptedException, IndexRecord rec = new IndexRecord(); final SpillRecord spillRec = new SpillRecord(partitions); + Collection start_context = XTraceContext.getThreadContext(); + Collection end_contexts = new XTraceMetadataCollection(); for (int parts = 0; parts < partitions; parts++) { + XTraceContext.setThreadContext(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 +1997,8 @@ private void mergeParts() throws IOException, InterruptedException, // sort the segments only if there are intermediate merges boolean sortSegments = segmentList.size() > mergeFactor; //merge + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer mergeParts", "Merging parts from multiple spills", + "Partition", parts, "Num Spills", numSpills); @SuppressWarnings("unchecked") RawKeyValueIterator kvIter = Merger.merge(job, rfs, keyClass, valClass, codec, @@ -1901,6 +2013,9 @@ private void mergeParts() throws IOException, InterruptedException, new Writer(job, finalOut, keyClass, valClass, codec, spilledRecordsCounter); if (combinerRunner == null || numSpills < minSpillsForCombine) { + XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer skipCombine", "Skipping combine", + "CombinerRunnerIsNull", combinerRunner==null, "MinSpillsForCombine", minSpillsForCombine, + "Partition", parts, "Num Spills", numSpills); Merger.writeFile(kvIter, writer, reporter, job); } else { combineCollector.setWriter(writer); @@ -1916,13 +2031,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 = XTraceContext.getThreadContext(end_contexts); } + XTraceContext.setThreadContext(end_contexts); spillRec.writeToFile(finalIndexFile, job); finalOut.close(); for(int i = 0; i < numSpills; i++) { rfs.delete(filename[i],true); } + XTraceContext.logEvent(MapTask.class, "MapTask", "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..87f9202dec62 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 @@ -43,6 +43,8 @@ import org.apache.hadoop.util.Progress; import org.apache.hadoop.util.Progressable; +import edu.berkeley.xtrace.XTraceContext; + /** * Merger is an utility class used by the Map and Reduce tasks for merging * both their memory and disk segments @@ -566,6 +568,8 @@ RawKeyValueIterator merge(Class keyClass, Class valueClass, Progress mergePhase) throws IOException { LOG.info("Merging " + segments.size() + " sorted segments"); + XTraceContext.logEvent(Merger.class, "Merger merge", "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..fa150e29d749 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 @@ -56,6 +56,8 @@ import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.ReflectionUtils; +import edu.berkeley.xtrace.XTraceContext; + /** A Reduce task. */ @InterfaceAudience.Private @InterfaceStability.Unstable @@ -311,6 +313,9 @@ private void writeSkippedRec(KEY key, VALUE value) throws IOException{ @SuppressWarnings("unchecked") public void run(JobConf job, final TaskUmbilicalProtocol umbilical) throws IOException, InterruptedException, ClassNotFoundException { + + XTraceContext.logEvent(ReduceTask.class, "ReduceTask", "ReduceTask running"); + job.setBoolean(JobContext.SKIP_RECORDS, isSkipping()); if (isMapOrReduce()) { @@ -641,10 +646,12 @@ public boolean next() throws IOException { committer, reporter, comparator, keyClass, valueClass); + XTraceContext.logEvent(ReduceTask.class, "NewReducer", "Running reduce start"); try { reducer.run(reducerContext); } finally { trackedRW.close(reducerContext); + XTraceContext.logEvent(ReduceTask.class, "NewReducer", "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..99b5b08c9b6e 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 @@ -38,6 +38,9 @@ import org.apache.hadoop.io.SecureIOUtils; import org.apache.hadoop.util.PureJavaCrc32; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + @InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceStability.Unstable public class SpillRecord { @@ -106,7 +109,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 +120,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..4de47aebdf39 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,6 +25,7 @@ import java.lang.management.ManagementFactory; import java.text.NumberFormat; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -69,6 +70,10 @@ import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.util.StringUtils; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + /** * Base class for tasks. */ @@ -139,6 +144,18 @@ static synchronized String getOutputName(int partition) { return "part-" + NUMBER_FORMAT.format(partition); } + //////////////////////////////////////////// + // X-Trace Methods + //////////////////////////////////////////// + private Collection xtrace_context = new XTraceMetadataCollection(); + public void rememberContext() { + XTraceContext.getThreadContext(xtrace_context); + } + public void joinContext() { + XTraceContext.joinContext(xtrace_context); + } + + //////////////////////////////////////////// // Fields //////////////////////////////////////////// @@ -321,6 +338,7 @@ protected void setWriteSkipRecs(boolean writeSkipRecs) { protected void reportFatalError(TaskAttemptID id, Throwable throwable, String logMsg) { LOG.fatal(logMsg); + XTraceContext.logEvent(Task.class, "FatalError", "Fatal error occurred", "Message", logMsg); Throwable tCause = throwable.getCause(); String cause = tCause == null ? StringUtils.stringifyException(throwable) @@ -329,6 +347,8 @@ protected void reportFatalError(TaskAttemptID id, Throwable throwable, umbilical.fatalError(id, cause); } catch (IOException ioe) { LOG.fatal("Failed to contact the tasktracker", ioe); + XTraceContext.logEvent(Task.class, "FatalError", "Failed to contact the tasktracker", "Exit Code", -1); + XTraceContext.joinParentProcess(); System.exit(-1); } } @@ -617,11 +637,13 @@ public class TaskReporter * Using AtomicBoolean since we need an atomic read & reset method. */ private AtomicBoolean progressFlag = new AtomicBoolean(false); + private Collection xtrace; TaskReporter(Progress taskProgress, TaskUmbilicalProtocol umbilical) { this.umbilical = umbilical; this.taskProgress = taskProgress; + this.xtrace = XTraceContext.getThreadContext(); } // getters and setters for flag @@ -701,6 +723,7 @@ public InputSplit getInputSplit() throws UnsupportedOperationException { * let the parent know that it's alive. It also pings the parent to see if it's alive. */ public void run() { + XTraceContext.setThreadContext(xtrace); final int MAX_RETRIES = 3; int remainingRetries = MAX_RETRIES; // get current flag value and reset it as well @@ -740,7 +763,9 @@ public void run() { // came back up), kill ourselves if (!taskFound) { LOG.warn("Parent died. Exiting "+taskId); + XTraceContext.logEvent(Task.class, "Task", "Parent died, exiting", "Exit Code", 66); resetDoneFlag(); + XTraceContext.joinParentProcess(); System.exit(66); } @@ -750,10 +775,14 @@ public void run() { catch (Throwable t) { LOG.info("Communication exception: " + StringUtils.stringifyException(t)); remainingRetries -=1; + XTraceContext.logEvent(Task.class, "Task", "Communication exception "+t.getClass().getName(), + "Message", t.getMessage(), "Retries Remaining", remainingRetries); if (remainingRetries == 0) { ReflectionUtils.logThreadInfo(LOG, "Communication exception", 0); LOG.warn("Last retry, killing "+taskId); + XTraceContext.logEvent(Task.class, "Task", "No retries remaining, killing task", "Exit Code", 65); resetDoneFlag(); + 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"); + + XTraceContext.logEvent(Task.class, "Task committing", "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)); + XTraceContext.logEvent(Task.class, "Task committing fail", "Failure sending commit pending: "+ie.getClass().getName(), + "Message", ie.getMessage(), "Retries Remaining", retries); if (--retries == 0) { + XTraceContext.logEvent(Task.class, "Task exiting", "No retries remaining for task commit, killing task", "Exit Code", 67); + XTraceContext.joinParentProcess(); System.exit(67); } } @@ -1056,6 +1091,8 @@ public void statusUpdate(TaskUmbilicalProtocol umbilical) try { if (!umbilical.statusUpdate(getTaskID(), taskStatus)) { LOG.warn("Parent died. Exiting "+taskId); + XTraceContext.logEvent(Task.class, "Task exiting", "Parent died, exiting", "Exit Code", 66); + XTraceContext.joinParentProcess(); System.exit(66); } taskStatus.clearStatus(); @@ -1111,12 +1148,16 @@ private void sendDone(TaskUmbilicalProtocol umbilical) throws IOException { int retries = MAX_RETRIES; while (true) { try { + XTraceContext.logEvent(Task.class, "Task done", "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)); + XTraceContext.logEvent(Task.class, "Task done signalling failure", + "Failure signalling completion: "+ie.getClass().getName(), "Message", ie.getMessage(), + "Retries Remaining", retries); if (--retries == 0) { throw ie; } @@ -1129,6 +1170,7 @@ private void commit(TaskUmbilicalProtocol umbilical, org.apache.hadoop.mapreduce.OutputCommitter committer ) throws IOException { int retries = MAX_RETRIES; + XTraceContext.logEvent(Task.class, "Await commit approval", "Awaiting commit approval from AM"); while (true) { try { while (!umbilical.canCommit(taskId)) { @@ -1143,9 +1185,13 @@ private void commit(TaskUmbilicalProtocol umbilical, } catch (IOException ie) { LOG.warn("Failure asking whether task can commit: " + StringUtils.stringifyException(ie)); + XTraceContext.logEvent(Task.class, "Commit approval failure", "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); + XTraceContext.logEvent(Task.class, "Commit approval exiting", "Maximum retries reached, discarding output and exiting", "Exit Code", 68); + XTraceContext.joinParentProcess(); System.exit(68); } } @@ -1154,11 +1200,14 @@ private void commit(TaskUmbilicalProtocol umbilical, // task can Commit now try { LOG.info("Task " + taskId + " is allowed to commit now"); + XTraceContext.logEvent(Task.class, "Commit approved", "Task is allowed to commit, committing"); committer.commitTask(taskContext); return; } catch (IOException iee) { LOG.warn("Failure committing: " + StringUtils.stringifyException(iee)); + XTraceContext.logEvent(Task.class, "Commit failed", "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 +1238,7 @@ void taskCleanup(TaskUmbilicalProtocol umbilical) getProgress().setStatus("cleanup"); statusUpdate(umbilical); LOG.info("Runnning cleanup for the task"); + XTraceContext.logEvent(Task.class, "TaskCleanup", "Runnning cleanup for the task"); // do the cleanup committer.abortTask(taskContext); } @@ -1202,9 +1252,11 @@ protected void runJobCleanupTask(TaskUmbilicalProtocol umbilical, statusUpdate(umbilical); // do the cleanup LOG.info("Cleaning up job"); + XTraceContext.logEvent(Task.class, "JobCleanupTask", "Cleaning up job"); if (jobRunStateForCleanup == JobStatus.State.FAILED || jobRunStateForCleanup == JobStatus.State.KILLED) { LOG.info("Aborting job with runstate : " + jobRunStateForCleanup.name()); + XTraceContext.logEvent(Task.class, "JobCleanupTask", "Aborting job with runstate : " + jobRunStateForCleanup.name()); if (conf.getUseNewMapper()) { committer.abortJob(jobContext, jobRunStateForCleanup); } else { @@ -1213,9 +1265,15 @@ protected void runJobCleanupTask(TaskUmbilicalProtocol umbilical, oldCommitter.abortJob(jobContext, jobRunStateForCleanup); } } else if (jobRunStateForCleanup == JobStatus.State.SUCCEEDED){ + XTraceContext.logEvent(Task.class, "JobCleanupTask", "Committing job"); LOG.info("Committing job"); committer.commitJob(jobContext); } else { + XTraceContext.logEvent(Task.class, "JobCleanupTask", "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 +1699,10 @@ public void combine(RawKeyValueIterator iterator, committer, reporter, comparator, keyClass, valueClass); + XTraceContext.logEvent(NewCombinerRunner.class, "Combiner start", "Combining map outputs", + "Combiner", reducer.getClass().getName(), "KeyClass", keyClass, "ValClass", valueClass); reducer.run(reducerContext); + XTraceContext.logEvent(NewCombinerRunner.class, "Combiner end", "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..f49f91462b38 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 @@ -32,7 +32,11 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + import java.io.IOException; +import java.util.Collection; import java.util.concurrent.*; /** @@ -205,6 +209,7 @@ private class MapperInvokeRunable implements Runnable { private V1 value; private OutputCollector output; private Reporter reporter; + private Collection xtrace_context; /** * Collecting all required parameters to execute a Mapper.map call. @@ -222,6 +227,7 @@ public MapperInvokeRunable(K1 key, V1 value, this.value = value; this.output = output; this.reporter = reporter; + this.xtrace_context = XTraceContext.getThreadContext(); } /** @@ -231,6 +237,7 @@ public MapperInvokeRunable(K1 key, V1 value, * */ public void run() { + XTraceContext.setThreadContext(xtrace_context); try { // map pair to output MultithreadedMapRunner.this.mapper.map(key, value, output, reporter); @@ -257,6 +264,7 @@ public void run() { } } } + XTraceContext.clearThreadContext(); } } 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..b58cb1b71fe0 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 @@ -42,6 +42,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; +import edu.berkeley.xtrace.XTraceContext; /** * Provides a way to access information about the map/reduce cluster. @@ -75,10 +76,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..4a2a4b6d5ba4 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,8 +19,10 @@ package org.apache.hadoop.mapreduce; import java.io.IOException; +import java.io.StringWriter; import java.net.URI; import java.security.PrivilegedExceptionAction; +import java.util.Collection; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -39,6 +41,9 @@ import org.apache.hadoop.mapreduce.util.ConfigUtil; import org.apache.hadoop.util.StringUtils; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * The job submitter's view of the Job. * @@ -110,6 +115,7 @@ public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL } private JobState state = JobState.DEFINE; private JobStatus status; + private Collection 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())) { + XTraceContext.startTrace("Hadoop Job", "Initializing Job", conf.getJobName()); + } else { + XTraceContext.startTrace("Hadoop Job", "Initializing Job"); + } } Job(JobStatus status, JobConf conf) throws IOException { @@ -307,6 +322,8 @@ synchronized void ensureFreshStatus() * @throws IOException */ synchronized void updateStatus() throws IOException { + Collection start_context = XTraceContext.getThreadContext(); + XTraceContext.clearThreadContext(); 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 = XTraceContext.getThreadContext(); + XTraceContext.setThreadContext(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() { + XTraceContext.joinContext(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 { + + XTraceContext.startTrace("MapReduce Job", "Preparing Job"); if (state == JobState.DEFINE) { submit(); } + XTraceContext.logEvent(Job.class, "MapReduce Job", "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()) { + XTraceContext.logEvent(Job.class, "MapReduce Job", "Job finished successfully", "Tag", maps+" Maps", "Tag", reds+" Reduces"); + } else { + XTraceContext.logEvent(Job.class, "MapReduce Job", "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"); + XTraceContext.logEvent(Job.class, "Job", "Job completed successfully"); } else { LOG.info("Job " + jobId + " failed with state " + status.getState() + " due to: " + status.getFailureInfo()); + XTraceContext.logEvent(Job.class, "Job", "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..001574e31cbc 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 @@ -30,7 +30,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; - +import edu.berkeley.xtrace.XTraceContext; /** * 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..2f2c7f80bd5d 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 @@ -25,6 +25,7 @@ import java.net.UnknownHostException; import java.security.NoSuchAlgorithmException; import java.util.Arrays; +import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -62,6 +63,9 @@ import org.codehaus.jackson.map.ObjectMapper; import com.google.common.base.Charsets; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceProcess; @InterfaceAudience.Private @InterfaceStability.Unstable @@ -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 + XTraceContext.logEvent(JobSubmitter.class, "JobSubmitter", "Submitting Job"); + Collection start_context = XTraceContext.getThreadContext(); + + //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); } + + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(JobSubmitter.class, "JobSubmitter", "Job submission complete"); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java index 384524a92ddf..16bace9c4052 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Map; @@ -41,6 +42,9 @@ import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer; import org.apache.hadoop.util.ReflectionUtils; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * The Chain class provides all the common functionality for the * {@link ChainMapper} and the {@link ChainReducer} classes. @@ -301,6 +305,7 @@ private class MapRunner extends Thread { private Mapper.Context chainContext; private RecordReader rr; private RecordWriter rw; + private Collection xtrace_context; public MapRunner(Mapper mapper, Mapper.Context mapperContext, @@ -310,10 +315,12 @@ public MapRunner(Mapper mapper, this.rr = rr; this.rw = rw; this.chainContext = mapperContext; + this.xtrace_context = XTraceContext.getThreadContext(); } @Override public void run() { + XTraceContext.joinContext(xtrace_context); if (getThrowable() != null) { return; } @@ -333,6 +340,7 @@ private class ReduceRunner extends Thread { private Reducer reducer; private Reducer.Context chainContext; private RecordWriter rw; + private Collection xtrace_context; ReduceRunner(Reducer.Context context, Reducer reducer, @@ -341,10 +349,12 @@ private class ReduceRunner extends Thread { this.reducer = reducer; this.chainContext = context; this.rw = rw; + this.xtrace_context = XTraceContext.getThreadContext(); } @Override public void run() { + XTraceContext.joinContext(xtrace_context); try { reducer.run(chainContext); rw.close(chainContext); 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..ece83735d8c6 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 @@ -36,8 +36,12 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; /** @@ -251,6 +255,7 @@ private class MapRunner extends Thread { private Context subcontext; private Throwable throwable; private RecordReader reader = new SubMapRecordReader(); + private Collection xtrace_context; MapRunner(Context context) throws IOException, InterruptedException { mapper = ReflectionUtils.newInstance(mapClass, @@ -265,10 +270,12 @@ private class MapRunner extends Thread { outer.getInputSplit()); subcontext = new WrappedMapper().getMapContext(mapContext); reader.initialize(context.getInputSplit(), context); + this.xtrace_context = XTraceContext.getThreadContext(); } @Override public void run() { + XTraceContext.joinContext(xtrace_context); try { mapper.run(subcontext); reader.close(); 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..7c9a0d622186 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,6 +48,10 @@ import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + class Fetcher extends Thread { private static final Log LOG = LogFactory.getLog(Fetcher.class); @@ -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 Collection initial_xtrace_context; + private Collection copy_contexts; + private static boolean sslShuffle; private static SSLFactory sslFactory; @@ -147,6 +155,8 @@ public Fetcher(JobConf job, TaskAttemptID reduceId, } } } + + initial_xtrace_context = XTraceContext.getThreadContext(); } public void run() { @@ -156,6 +166,8 @@ public void run() { try { // If merge is on, block merger.waitForResource(); + + XTraceContext.setThreadContext(initial_xtrace_context); // Get a host to shuffle from host = scheduler.getHost(); @@ -168,6 +180,8 @@ public void run() { scheduler.freeHost(host); metrics.threadFree(); } + copy_contexts = XTraceContext.getThreadContext(copy_contexts); + XTraceContext.clearThreadContext(); } } } catch (InterruptedException ie) { @@ -198,6 +212,10 @@ public void shutDown() throws InterruptedException { sslFactory.destroy(); } } + + public void joinContexts() { + XTraceContext.joinContext(copy_contexts); + } @VisibleForTesting protected synchronized void openConnection(URL url) @@ -251,6 +269,8 @@ protected void copyFromHost(MapHost host) throws IOException { LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: " + maps); } + XTraceContext.logEvent(Fetcher.class, "Fetcher", "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); + XTraceContext.logEvent(Fetcher.class, "Fetcher", "Connecting to map output on host", "URL", url); openConnection(url); if (stopped) { abortConnect(host, remaining); @@ -273,6 +294,7 @@ protected void copyFromHost(MapHost host) throws IOException { // put url hash into http header connection.addRequestProperty( SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash); + connection.addRequestProperty("X-Trace", XTraceContext.logMerge().toString()); // set the read timeout connection.setReadTimeout(readTimeout); // put shuffle version into http header @@ -291,6 +313,12 @@ 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) { + XTraceContext.joinContext(XTraceMetadata.createFromString(xtrace_context)); + } + XTraceContext.logEvent(Fetcher.class, "Fetcher", "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 +344,8 @@ protected void copyFromHost(MapHost host) throws IOException { ioErrs.increment(1); LOG.warn("Failed to connect to " + host + " with " + remaining.size() + " map outputs", ie); + XTraceContext.logEvent(Fetcher.class, "Fetcher","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; + Collection start_context = XTraceContext.getThreadContext(); + Collection end_contexts = new XTraceMetadataCollection(); + int initialSize = remaining.size(); while (!remaining.isEmpty() && failedTasks == null) { + XTraceContext.setThreadContext(start_context); failedTasks = copyMapOutput(host, input, remaining); + end_contexts = XTraceContext.getThreadContext(end_contexts); } + XTraceContext.joinContext(end_contexts); if(failedTasks != null && failedTasks.length > 0) { LOG.warn("copyMapOutput failed for tasks "+Arrays.toString(failedTasks)); + XTraceContext.logEvent(Fetcher.class, "Fetcher", "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,18 @@ protected void copyFromHost(MapHost host) throws IOException { // Sanity check if (failedTasks == null && !remaining.isEmpty()) { + XTraceContext.logEvent(Fetcher.class, "Fetcher", "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; + XTraceContext.logEvent(Fetcher.class, "Fetching complete", "Fetching complete", + "Num Succeeded", copied, "Num Failed", failed); + input.close(); input = null; } finally { @@ -383,6 +430,8 @@ private TaskAttemptID[] copyMapOutput(MapHost host, try { ShuffleHeader header = new ShuffleHeader(); header.readFields(input); + XTraceContext.clearThreadContext(); + header.joinContext(); mapId = TaskAttemptID.forName(header.mapId); compressedLength = header.compressedLength; decompressedLength = header.uncompressedLength; @@ -390,6 +439,8 @@ private TaskAttemptID[] copyMapOutput(MapHost host, } catch (IllegalArgumentException e) { badIdErrs.increment(1); LOG.warn("Invalid map id ", e); + XTraceContext.logEvent(Fetcher.class, "Fetcher", "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 +458,13 @@ private TaskAttemptID[] copyMapOutput(MapHost host, } // Get the location for the map output - either in-memory or on-disk + XTraceContext.logEvent(Fetcher.class, "Fetcher", "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 ..."); + XTraceContext.logEvent(Fetcher.class, "Fetcher", "Merge Manager instructed fetcher to wait", + "Fetcher ID", id); //Not an error but wait to process data. return EMPTY_ATTEMPT_ID_ARRAY; } @@ -424,6 +477,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()); + XTraceContext.logEvent(Fetcher.class, "Fetcher", "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 +501,9 @@ private TaskAttemptID[] copyMapOutput(MapHost host, LOG.info("fetcher#" + id + " failed to read map header" + mapId + " decomp: " + decompressedLength + ", " + compressedLength, ioe); + XTraceContext.logEvent(Fetcher.class, "Fetcher", "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 +513,8 @@ private TaskAttemptID[] copyMapOutput(MapHost host, LOG.warn("Failed to shuffle output of " + mapId + " from " + host.getHostName(), ioe); + XTraceContext.logEvent(Fetcher.class, "Fetcher", "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..b33f0efbd525 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 @@ -40,6 +40,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptID; +import edu.berkeley.xtrace.XTraceContext; + @InterfaceAudience.Private @InterfaceStability.Unstable class InMemoryMapOutput extends MapOutput { @@ -125,6 +127,7 @@ public void shuffle(MapHost host, InputStream input, @Override public void commit() throws IOException { + XTraceContext.logEvent(InMemoryMapOutput.class, "InMemoryMapOutput", "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..202b595cb1d9 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 @@ -20,6 +20,7 @@ import java.io.InputStream; import java.io.IOException; +import java.util.Collection; import java.util.Comparator; import java.util.concurrent.atomic.AtomicInteger; @@ -30,6 +31,9 @@ import org.apache.hadoop.mapreduce.TaskAttemptID; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + @InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceStability.Unstable public abstract class MapOutput { @@ -40,6 +44,8 @@ public abstract class MapOutput { private final long size; private final boolean primaryMapOutput; + private Collection xtrace_context; + public MapOutput(TaskAttemptID mapId, long size, boolean primaryMapOutput) { this.id = ID.incrementAndGet(); this.mapId = mapId; @@ -47,6 +53,14 @@ public MapOutput(TaskAttemptID mapId, long size, boolean primaryMapOutput) { this.primaryMapOutput = primaryMapOutput; } + public void rememberContext() { + xtrace_context = XTraceContext.getThreadContext(); + } + + public void joinContext() { + XTraceContext.joinContext(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..bda61dc227c3 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 @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -60,6 +61,9 @@ import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + @SuppressWarnings(value={"unchecked"}) @InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceStability.Unstable @@ -254,6 +258,9 @@ public synchronized MapOutput reserve(TaskAttemptID mapId, LOG.info(mapId + ": Shuffling to disk since " + requestedSize + " is greater than maxSingleShuffleLimit (" + maxSingleShuffleLimit + ")"); + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl", + "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 +283,10 @@ 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 + ")"); + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl", + "Currently above memory limit, stalling shuffle", "Map ID", mapId, + "Used Memory", usedMemory, "Memory Limit", memoryLimit, "Commit Memory", commitMemory); return null; } @@ -284,6 +294,9 @@ public synchronized MapOutput reserve(TaskAttemptID mapId, LOG.debug(mapId + ": Proceeding with shuffle since usedMemory (" + usedMemory + ") is lesser than memoryLimit (" + memoryLimit + ")." + "CommitMemory is (" + commitMemory + ")"); + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl", + "Proceeding with shuffle", "Map ID", mapId, "Used Memory", usedMemory, + "Memory Limit", memoryLimit, "Commit Memory", commitMemory); return unconditionalReserve(mapId, requestedSize, true); } @@ -302,7 +315,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 +344,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 +352,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 +407,9 @@ public void merge(List> inputs) throws IOException { LOG.info("Initiating Memory-to-Memory merge with " + noInMemorySegments + " segments of total-size: " + mergeOutputSize); + XTraceContext.logEvent(IntermediateMemoryToMemoryMerger.class, "IntermediateMemoryToMemoryMerger start", + "Initiating Memory-to-Memory merge", "Num Segments", noInMemorySegments, "Total Size", mergeOutputSize, + "Reduce ID", reduceId); RawKeyValueIterator rIter = Merger.merge(jobConf, rfs, @@ -406,6 +425,8 @@ public void merge(List> inputs) throws IOException { LOG.info(reduceId + " Memory-to-Memory merge of the " + noInMemorySegments + " files in-memory complete."); + XTraceContext.logEvent(IntermediateMemoryToMemoryMerger.class, "IntermediateMemoryToMemoryMerger end", + "Memory-to-Memory merge complete", "Reduce ID", reduceId, "Num Segments", noInMemorySegments); // Note the output of the merge closeInMemoryMergedFile(mergedMapOutputs); @@ -457,9 +478,12 @@ public void merge(List> inputs) throws IOException { RawKeyValueIterator rIter = null; CompressAwarePath compressAwarePath; + try { LOG.info("Initiating in-memory merge with " + noInMemorySegments + " segments..."); + XTraceContext.logEvent(InMemoryMerger.class, "InMemoryMerger start", "Initiating in-memory merge", + "Num Segments", noInMemorySegments); rIter = Merger.merge(jobConf, rfs, (Class)jobConf.getMapOutputKeyClass(), @@ -484,12 +508,16 @@ public void merge(List> inputs) throws IOException { " files in-memory complete." + " Local file is " + outputPath + " of size " + localFS.getFileStatus(outputPath).getLen()); + XTraceContext.logEvent(InMemoryMerger.class, "InMemoryMerger","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; } + + XTraceContext.logEvent(InMemoryMerger.class, "InMemoryMerger end", "In-memory merge finished"); // Note the output of the merge closeOnDiskFile(compressAwarePath); @@ -512,13 +540,20 @@ public void merge(List inputs) throws IOException { LOG.info("No ondisk files to merge..."); return; } + for (CompressAwarePath input : inputs) { + XTraceContext.joinObject(input); + } 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..."); + XTraceContext.logEvent(OnDiskMerger.class, "OnDiskMerger start", + "Merging outputs on disk", "Num Segments", inputs.size()); // 1. Prepare the list of files to be merged. for (CompressAwarePath file : inputs) { @@ -559,6 +594,9 @@ public void merge(List inputs) throws IOException { localFS.delete(outputPath, true); throw e; } + + XTraceContext.logEvent(OnDiskMerger.class, "OnDiskMerger merge end", "Finished merging map output files on disk", + "Num Segments", inputs.size(), "Output Size", approxOutputSize, "Output File", outputPath); closeOnDiskFile(compressAwarePath); @@ -607,6 +645,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 +703,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"); + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge", "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 +728,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, List> memDiskSegments = new ArrayList>(); long inMemToDiskBytes = 0; boolean mergePhaseFinished = false; + Collection start_context = XTraceContext.getThreadContext(); if (inMemoryMapOutputs.size() > 0) { TaskID mapId = inMemoryMapOutputs.get(0).getMapId().getTaskID(); inMemToDiskBytes = createInMemorySegments(inMemoryMapOutputs, @@ -695,7 +737,9 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, final int numMemDiskSegments = memDiskSegments.size(); if (numMemDiskSegments > 0 && ioSortFactor > onDiskMapOutputs.size()) { - + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge flush", + "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 +761,13 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, keyClass, valueClass, codec, null); try { Merger.writeFile(rIter, writer, reporter, job); + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge flush merge", "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); + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge flush complete", "Merged to disk", + "OutputPath", outputPath.toString()); + p.rememberContext(); writer = null; // add to list of final disk outputs. } catch (IOException e) { @@ -741,6 +789,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, "reduce memory limit"); inMemToDiskBytes = 0; memDiskSegments.clear(); + XTraceContext.setThreadContext(start_context); } else if (inMemToDiskBytes != 0) { LOG.info("Keeping " + numMemDiskSegments + " segments, " + inMemToDiskBytes + " bytes in memory for " + @@ -748,6 +797,8 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, } } + start_context = XTraceContext.getThreadContext(); + // segments on disk List> diskSegments = new ArrayList>(); long onDiskBytes = inMemToDiskBytes; @@ -755,6 +806,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 +820,8 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, } LOG.info("Merging " + onDisk.length + " files, " + onDiskBytes + " bytes from disk"); + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge", "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 +830,9 @@ public int compare(Segment o1, Segment o2) { return o1.getLength() < o2.getLength() ? -1 : 1; } }); + + Collection disk_context = XTraceContext.getThreadContext(); + XTraceContext.setThreadContext(start_context); // build final list of segments from merged backed by disk + in-mem List> finalSegments = new ArrayList>(); @@ -783,7 +840,11 @@ public int compare(Segment o1, Segment o2) { finalSegments, 0); LOG.info("Merging " + finalSegments.size() + " segments, " + inMemBytes + " bytes from memory into reduce"); + + Collection mem_context = XTraceContext.getThreadContext(); + if (0 != onDiskBytes) { + XTraceContext.setThreadContext(disk_context); final int numInMemSegments = memDiskSegments.size(); diskSegments.addAll(0, memDiskSegments); memDiskSegments.clear(); @@ -795,28 +856,45 @@ public int compare(Segment o1, Segment o2) { ioSortFactor, numInMemSegments, tmpDir, comparator, reporter, false, spilledRecordsCounter, null, thisPhase); diskSegments.clear(); + XTraceContext.joinContext(mem_context); + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge","Piping segment merge into reduce", + "Num Segments", finalSegments.size(), "Num Bytes", inMemBytes); if (0 == finalSegments.size()) { + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge complete", + "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); - + + XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge complete", + "Final merge complete, returning result iterator"); + return result; } static class CompressAwarePath extends Path { private long rawDataLength; private long compressedSize; + private Collection xtrace_context; public CompressAwarePath(Path path, long rawDataLength, long compressSize) { super(path.toUri()); this.rawDataLength = rawDataLength; this.compressedSize = compressSize; } + + public void rememberContext() { + xtrace_context = XTraceContext.getThreadContext(); + } + + public void joinContext() { + XTraceContext.joinContext(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..cbf831ddef6a 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,6 +28,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import edu.berkeley.xtrace.XTraceContext; + abstract class MergeThread extends Thread { private static final Log LOG = LogFactory.getLog(MergeThread.class); @@ -72,8 +74,11 @@ public void startMerge(Set inputs) { } public synchronized void waitForMerge() throws InterruptedException { - while (numPending.get() > 0) { + int numPending = this.numPending.get(); + while (numPending > 0) { + XTraceContext.logEvent(MergeThread.class, "MergeThread", "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..2e6256f16a93 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 @@ -40,6 +40,7 @@ import org.apache.hadoop.mapreduce.task.reduce.MergeManagerImpl.CompressAwarePath; import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; @InterfaceAudience.Private @InterfaceStability.Unstable @@ -134,6 +135,8 @@ public void commit() throws IOException { CompressAwarePath compressAwarePath = new CompressAwarePath(outputPath, getSize(), this.compressedSize); merger.closeOnDiskFile(compressAwarePath); + XTraceContext.logEvent(OnDiskMapOutput.class, "OnDiskMapOutput", "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..762ecc3cea74 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.task.reduce; import java.io.IOException; +import java.util.Collection; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -32,6 +33,9 @@ import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.util.Progress; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + @InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceStability.Unstable @SuppressWarnings({"unchecked", "rawtypes"}) @@ -53,6 +57,7 @@ public class Shuffle implements ShuffleConsumerPlugin, ExceptionRepo private MergeManager merger; private Throwable throwable = null; private String throwingThreadName = null; + private Collection throwingContext = null; private Progress copyPhase; private TaskStatus taskStatus; private Task reduceTask; //Used for status updates @@ -89,6 +94,8 @@ protected MergeManager createMergeManager( @Override public RawKeyValueIterator run() throws IOException, InterruptedException { + XTraceContext.logEvent(Shuffle.class, "Shuffle", "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 +125,28 @@ public RawKeyValueIterator run() throws IOException, InterruptedException { synchronized (this) { if (throwable != null) { + XTraceContext.joinContext(throwingContext); + XTraceContext.logEvent(Shuffle.class, "Shuffle", "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(); + XTraceContext.logEvent(Shuffle.class, "Shuffle copy complete", "Copy phase complete"); copyPhase.complete(); // copy is already complete taskStatus.setPhase(TaskStatus.Phase.SORT); reduceTask.statusUpdate(umbilical); @@ -144,17 +156,24 @@ public RawKeyValueIterator run() throws IOException, InterruptedException { try { kvIter = merger.close(); } catch (Throwable e) { + XTraceContext.logEvent(Shuffle.class, "Shuffle merge error", "Error during final merge: "+e.getClass().getName(), + "Message", e.getMessage()); throw new ShuffleError("Error while doing final merge " , e); } // Sanity check synchronized (this) { if (throwable != null) { + XTraceContext.joinContext(throwingContext); + XTraceContext.logEvent(Shuffle.class, "Shuffle", "Error during shuffle: "+throwable.getClass().getName(), + "Throwing Thread", throwingThreadName, "Message", throwable.getMessage()); throw new ShuffleError("error in shuffle in " + throwingThreadName, throwable); } } + XTraceContext.logEvent(Shuffle.class, "Shuffle complete", "Shuffle complete"); + return kvIter; } @@ -166,6 +185,7 @@ public synchronized void reportException(Throwable t) { if (throwable == null) { throwable = t; throwingThreadName = Thread.currentThread().getName(); + throwingContext = XTraceContext.getThreadContext(); // 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..85c7dbbcfe16 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 @@ -20,6 +20,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.nio.ByteBuffer; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -27,6 +28,10 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; +import edu.berkeley.xtrace.TaskID; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * Shuffle Header information that is sent by the TaskTracker and * deciphered by the Fetcher thread of Reduce task @@ -51,6 +56,7 @@ public class ShuffleHeader implements Writable { long uncompressedLength; long compressedLength; int forReduce; + XTraceMetadata m; public ShuffleHeader() { } @@ -67,6 +73,9 @@ public void readFields(DataInput in) throws IOException { compressedLength = WritableUtils.readVLong(in); uncompressedLength = WritableUtils.readVLong(in); forReduce = WritableUtils.readVInt(in); + byte[] taskid = ByteBuffer.allocate(8).putLong(WritableUtils.readVLong(in)).array(); + byte[] opid = ByteBuffer.allocate(8).putLong(WritableUtils.readVLong(in)).array(); + m = new XTraceMetadata(new TaskID(taskid, 8), opid); } public void write(DataOutput out) throws IOException { @@ -74,5 +83,36 @@ public void write(DataOutput out) throws IOException { WritableUtils.writeVLong(out, compressedLength); WritableUtils.writeVLong(out, uncompressedLength); WritableUtils.writeVInt(out, forReduce); + WritableUtils.writeVLong(out, getXTraceTaskID()); + WritableUtils.writeVLong(out, getXTraceOpID()); + } + + private long getXTraceTaskID() { + if (m!=null) { + try { + return ByteBuffer.wrap(m.getTaskId().get()).getLong(); + } catch (Exception e) { + } + } + return 0L; + } + + private long getXTraceOpID() { + if (m!=null) { + try { + return ByteBuffer.wrap(m.getOpId()).getLong(); + } catch (Exception e) { + } + } + return 0L; + } + + + public void rememberContext() { + m = XTraceContext.logMerge(); + } + + public void joinContext() { + XTraceContext.joinChildProcess(m); } } \ 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..dc2961ceba4d 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 @@ -22,6 +22,7 @@ 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 +49,10 @@ import org.apache.hadoop.mapreduce.task.reduce.MapHost.State; import org.apache.hadoop.util.Progress; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + @InterfaceAudience.Private @InterfaceStability.Unstable public class ShuffleSchedulerImpl implements ShuffleScheduler { @@ -100,6 +105,8 @@ protected Long initialValue() { private final boolean reportReadErrorImmediately; private long maxDelay = MRJobConfig.DEFAULT_MAX_SHUFFLE_FETCH_RETRY_DELAY; + private Collection failure_contexts = new XTraceMetadataCollection(); + public ShuffleSchedulerImpl(JobConf job, TaskStatus status, TaskAttemptID reduceId, ExceptionReporter reporter, @@ -195,6 +202,8 @@ public synchronized void copySucceeded(TaskAttemptID mapId, reduceShuffleBytes.increment(bytes); lastProgressTime = System.currentTimeMillis(); LOG.debug("map " + mapId + " done " + status.getStateString()); + XTraceContext.logEvent(ShuffleScheduler.class, "ShuffleScheduler", + "Shuffle from mapper complete", "Map ID", mapId, "Status", status.getStateString()); } } @@ -230,8 +239,10 @@ public synchronized void copyFailed(TaskAttemptID mapId, MapHost host, } else { hostFailures.put(hostname, new IntWritable(1)); } + failure_contexts = XTraceContext.getThreadContext(failure_contexts); if (failures >= abortFailureLimit) { try { + XTraceContext.joinContext(failure_contexts); throw new IOException(failures + " failures downloading " + mapId); } catch (IOException ie) { reporter.reportException(ie); @@ -368,6 +379,8 @@ public synchronized MapHost getHost() throws InterruptedException { LOG.info("Assigning " + host + " with " + host.getNumKnownMapOutputs() + " to " + Thread.currentThread().getName()); + XTraceContext.logEvent(ShuffleScheduler.class, "ShuffleScheduler", "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/HistoryFileManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java index a88e8549427f..6d66c3852e31 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java @@ -67,6 +67,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * This class provides a way to interact with history files in a thread safe * manor. @@ -745,15 +748,18 @@ private void scanIntermediateDirectory(final Path absPath) throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("Scheduling move to done of " +found); } + final Collection xtrace_context = XTraceContext.getThreadContext(); moveToDoneExecutor.execute(new Runnable() { @Override public void run() { + XTraceContext.setThreadContext(xtrace_context); try { found.moveToDone(); } catch (IOException e) { LOG.info("Failed to process fileInfo for job: " + found.getJobId(), e); } + XTraceContext.clearThreadContext(); } }); } 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..176dd4cb61d3 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.mapreduce.v2.hs; import java.io.IOException; +import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -52,6 +53,9 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * Loads and manages the Job history cache. */ @@ -172,19 +176,36 @@ public String getApplicationName() { } private class MoveIntermediateToDoneRunnable implements Runnable { + + private Collection xtrace_context; + + public MoveIntermediateToDoneRunnable() { + this.xtrace_context = XTraceContext.getThreadContext(); + } + @Override public void run() { + XTraceContext.setThreadContext(xtrace_context); try { LOG.info("Starting scan to move intermediate done files"); hsManager.scanIntermediateDirectory(); } catch (IOException e) { LOG.error("Error while scanning intermediate done dir ", e); } + XTraceContext.clearThreadContext(); } } private class HistoryCleaner implements Runnable { + + private Collection xtrace_context; + + public HistoryCleaner() { + this.xtrace_context = XTraceContext.getThreadContext(); + } + public void run() { + XTraceContext.setThreadContext(xtrace_context); LOG.info("History Cleaner started"); try { hsManager.clean(); @@ -192,6 +213,7 @@ public void run() { LOG.warn("Error trying to clean up ", e); } LOG.info("History Cleaner complete"); + XTraceContext.clearThreadContext(); } } 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..d38699e5f5a3 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 @@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.util.ConverterUtils; +import edu.berkeley.xtrace.XTraceContext; import com.google.common.annotations.VisibleForTesting; @@ -170,7 +171,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); + XTraceContext.logEvent("ResourceMgrDelegate", "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/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java index f9d1203ba4df..c8100185f697 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 @@ -40,6 +40,7 @@ import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -113,6 +114,8 @@ import com.google.common.base.Charsets; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; public class ShuffleHandler extends AuxiliaryService { @@ -446,6 +449,12 @@ 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) { + XTraceContext.setThreadContext(XTraceMetadata.createFromString(xtrace_context)); + } + final Map> q = new QueryStringDecoder(request.getUri()).getParameters(); final List mapIds = splitMaps(q.get("map")); @@ -457,6 +466,9 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) "\n reduceId: " + reduceQ + "\n jobId: " + jobQ); } + XTraceContext.logEvent(ShuffleHandler.class, "ShuffleHandler", + "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 +510,13 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) ch.write(response); // TODO refactor the following into the pipeline ChannelFuture lastMap = null; + Collection start_context = XTraceContext.getThreadContext(); for (String mapId : mapIds) { try { lastMap = sendMapOutput(ctx, ch, userRsrc.get(jobId), jobId, mapId, reduceId); if (null == lastMap) { + XTraceContext.logEvent(ShuffleHandler.class, "ShuffleHandler", "Error: "+NOT_FOUND); sendError(ctx, NOT_FOUND); return; } @@ -514,9 +528,12 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) sb.append(t.getCause().getMessage()); t = t.getCause(); } + XTraceContext.logEvent(ShuffleHandler.class, "ShuffleHandler", "Shuffle error: "+e.getClass().getName(), + "Message", e.getMessage()); sendError(ctx,sb.toString() , INTERNAL_SERVER_ERROR); return; } + XTraceContext.setThreadContext(start_context); } lastMap.addListener(metrics); lastMap.addListener(ChannelFutureListener.CLOSE); @@ -591,8 +608,12 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, } final IndexRecord info = indexCache.getIndexInformation(mapId, reduce, indexFileName, user); + info.joinContext(); + XTraceContext.logEvent(ShuffleHandler.class, "ShuffleHandler", + "Sending map output", "Reduce", reduce, "Map ID", mapId, "Job ID", jobID); final ShuffleHeader header = new ShuffleHeader(mapId, info.partLength, info.rawLength, reduce); + header.rememberContext(); final DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); @@ -647,6 +668,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", XTraceContext.logMerge()); // 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..8f3999b3cebc 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 @@ -31,6 +31,8 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.GenericOptionsParser; +import edu.berkeley.xtrace.XTraceContext; + public class WordCount { public static class TokenizerMapper @@ -64,12 +66,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 +83,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..c4e67773f70b 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.berkeley.xtrace.XTraceContext; + /** * This class is intended to be installed by calling * {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)} @@ -49,6 +51,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); + XTraceContext.logEvent(UncaughtExceptionHandler.class, "UncaughtError", 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 +61,19 @@ 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..."); + XTraceContext.joinParentProcess(); } catch (Throwable err) { //Again we done want to exit because of logging issues. } ExitUtil.halt(-1); } else { + XTraceContext.joinParentProcess(); ExitUtil.terminate(-1); } } else { LOG.error("Thread " + t + " threw an Exception.", e); + XTraceContext.logEvent(UncaughtExceptionHandler.class, "UncaughtException", 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..d0d67a0c4ac9 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 @@ -20,12 +20,15 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import com.google.protobuf.ByteString; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; import com.google.common.base.Preconditions; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; @Private @Unstable @@ -33,6 +36,7 @@ public class ContainerIdPBImpl extends ContainerId { ContainerIdProto proto = null; ContainerIdProto.Builder builder = null; private ApplicationAttemptId applicationAttemptId = null; + XTraceMetadata xmd = null; public ContainerIdPBImpl() { builder = ContainerIdProto.newBuilder(); @@ -41,6 +45,12 @@ public ContainerIdPBImpl() { public ContainerIdPBImpl(ContainerIdProto proto) { this.proto = proto; this.applicationAttemptId = convertFromProtoFormat(proto.getAppAttemptId()); + if (proto!=null && proto.hasXtrace()) { + ByteString xbs = proto.getXtrace(); + xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); + if (!xmd.isValid()) + xmd = null; + } } public ContainerIdProto getProto() { @@ -74,6 +84,21 @@ protected void setApplicationAttemptId(ApplicationAttemptId atId) { this.applicationAttemptId = atId; } + @Override + public void rememberContext() { + XTraceMetadata ctx = XTraceContext.logMerge(); + if (ctx!=null && ctx.isValid()) { + xmd = ctx; + } else { + xmd = null; + } + } + + @Override + public void joinContext() { + XTraceContext.joinContext(xmd); + } + private ApplicationAttemptIdPBImpl convertFromProtoFormat( ApplicationAttemptIdProto p) { return new ApplicationAttemptIdPBImpl(p); @@ -86,6 +111,9 @@ private ApplicationAttemptIdProto convertToProtoFormat( @Override protected void build() { + if (xmd!=null && xmd.isValid() && builder!=null) { + builder.setXtrace(ByteString.copyFrom(xmd.pack())); + } 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..93ca4c29fd3a 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 @@ -20,6 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import com.google.protobuf.ByteString; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; @@ -31,6 +32,8 @@ import org.apache.hadoop.yarn.proto.YarnProtos.URLProto; import com.google.protobuf.TextFormat; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; @Private @Unstable @@ -192,6 +195,27 @@ public synchronized void setPattern(String pattern) { builder.setPattern(pattern); } + @Override + public void rememberContext() { + maybeInitBuilder(); + XTraceMetadata ctx = XTraceContext.logMerge(); + if (ctx!=null && ctx.isValid()) { + builder.setXtrace(ByteString.copyFrom(ctx.pack())); + } + } + + @Override + public void joinContext() { + LocalResourceProtoOrBuilder p = viaProto ? proto : builder; + if (p.hasXtrace()) { + ByteString xbs = p.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); + if (xmd.isValid()) { + XTraceContext.joinContext(xmd); + } + } + } + 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..71200fa6d996 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 @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import com.google.protobuf.ByteString; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -29,6 +30,9 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProtoOrBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + @Private @Unstable public class ResourceRequestPBImpl extends ResourceRequest { @@ -163,6 +167,27 @@ public void setRelaxLocality(boolean relaxLocality) { builder.setRelaxLocality(relaxLocality); } + @Override + public void rememberContext() { + maybeInitBuilder(); + XTraceMetadata ctx = XTraceContext.logMerge(); + if (ctx!=null && ctx.isValid()) { + builder.setXtrace(ByteString.copyFrom(ctx.pack())); + } + } + + @Override + public void joinContext() { + ResourceRequestProtoOrBuilder p = viaProto ? proto : builder; + if (p.hasXtrace()) { + ByteString xbs = p.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); + if (xmd.isValid()) { + XTraceContext.joinContext(xmd); + } + } + } + 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..2e24cefe0ffe 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 @@ -20,6 +20,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; +import java.util.Collection; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; /** * 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 Collection 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 = XTraceContext.getThreadContext(); + } + + @Override + public void joinContext() { + XTraceContext.joinContext(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..48ec2637ec99 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,6 +19,7 @@ package org.apache.hadoop.yarn.event; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -34,6 +35,10 @@ import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + /** * Dispatches {@link Event}s in a separate thread. Currently only single thread * does that. Potentially there could be multiple channels for each event type @@ -68,6 +73,7 @@ Runnable createThread() { @Override public void run() { while (!stopped && !Thread.currentThread().isInterrupted()) { + XTraceContext.clearThreadContext(); Event event; try { event = eventQueue.take(); @@ -78,6 +84,7 @@ public void run() { return; } if (event != null) { + event.joinContext(); dispatch(event); } } @@ -120,6 +127,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 +150,8 @@ protected void dispatch(Event event) { if (exitOnDispatchException && (ShutdownHookManager.get().isShutdownInProgress()) == false) { LOG.info("Exiting, bbye.."); + XTraceContext.logEvent(AsyncDispatcher.class, "AsyncDispatcher", "Exiting, bbye.."); + XTraceContext.joinParentProcess(); System.exit(-1); } } @@ -213,9 +223,15 @@ public MultiListenerHandler() { @Override public void handle(Event event) { + Collection start_xtrace_context = XTraceContext.getThreadContext(); + Collection result_xtrace_contexts = new XTraceMetadataCollection(); for (EventHandler handler: listofHandlers) { + XTraceContext.setThreadContext(start_xtrace_context); + event.joinContext(); handler.handle(event); + result_xtrace_contexts = XTraceContext.getThreadContext(result_xtrace_contexts); } + XTraceContext.setThreadContext(result_xtrace_contexts); } 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/factories/impl/pb/RpcClientFactoryPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java index e7c737cbb489..f644a2aef532 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java @@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RpcClientFactory; +import edu.berkeley.xtrace.XTraceContext; @Private public class RpcClientFactoryPBImpl implements RpcClientFactory { 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..4f5620d91633 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.state; +import java.util.Collection; import java.util.EnumMap; import java.util.HashMap; import java.util.Iterator; @@ -28,6 +29,11 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; +import org.apache.hadoop.yarn.state.StateMachineFactory.Trace; + +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceEvent; /** * State machine topology. @@ -46,6 +52,8 @@ final public class StateMachineFactory , EVENTTYPE extends Enum, EVENT> { + + public enum Trace { NEVER, ALWAYS, KEEPALIVE, IGNORE } private final TransitionsListNode transitionsListNode; @@ -55,18 +63,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 +98,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 +112,7 @@ public StateMachineFactory(STATE defaultInitialState) { } else { stateMachineTable = null; } + this.xtrace_default = that.xtrace_default; } private interface ApplicableTransition @@ -160,7 +183,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 +224,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 +265,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 +315,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 +360,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 +434,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 && !XTraceContext.isValid()); + } + } + return false; + } private synchronized void maybeMakeStateMachineTable() { if (stateMachineTable == null) { @@ -341,6 +505,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 +514,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 +532,11 @@ public STATE doTransition(OPERAND operand, STATE oldState, } return postState; } + + @Override + public Trace getTraceType() { + return xtrace_process; + } } private class MultipleInternalArc @@ -371,11 +545,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 +566,11 @@ public STATE doTransition(OPERAND operand, STATE oldState, } return postState; } + + @Override + public Trace getTraceType() { + return xtrace_process; + } } /* @@ -406,7 +588,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,7 +603,11 @@ 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 @@ -429,24 +615,57 @@ private class InternalStateMachine private final OPERAND operand; private STATE currentState; - InternalStateMachine(OPERAND operand, STATE initialState) { + private String state_machine_id; + private Collection previous_transition_context; + + InternalStateMachine(OPERAND operand, STATE initialState, int xtrace_id_seed) { this.operand = operand; this.currentState = initialState; if (!optimized) { maybeMakeStateMachineTable(); } + this.state_machine_id = operand.getClass().getSimpleName()+"-"+xtrace_id_seed; + XTraceContext.logEvent(operand.getClass(), operand.getClass().getSimpleName()+" init", "StateMachine initialized", + "StartState", xtraceStateName(currentState), "StateMachineID", state_machine_id); + this.previous_transition_context = XTraceContext.getThreadContext(); } @Override public synchronized STATE getCurrentState() { return currentState; } - - @Override + + public void joinPreviousTransitionXTraceContext() { + XTraceContext.joinContext(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 + XTraceContext.logEvent(operand.getClass(), operand.getClass().getSimpleName()+event.toString(), 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 && XTraceContext.isValid()) { + this.previous_transition_context = XTraceContext.getThreadContext(); + } + 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..39522d913e7a 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 @@ -23,6 +23,8 @@ import java.io.IOException; import java.net.URISyntaxException; import java.security.PrivilegedExceptionAction; +import java.util.Collection; +import java.util.Random; import java.util.concurrent.Callable; import java.util.regex.Pattern; @@ -43,6 +45,9 @@ import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * Download a single URL to the local disk. * @@ -60,6 +65,8 @@ public class FSDownload implements Callable { /** The local FS dir path under which this resource is to be localized to */ private Path destDirPath; + private Collection xtrace_context; + private static final FsPermission cachePerms = new FsPermission( (short) 0755); static final FsPermission PUBLIC_FILE_PERMS = new FsPermission((short) 0555); @@ -76,6 +83,7 @@ public FSDownload(FileContext files, UserGroupInformation ugi, Configuration con this.files = files; this.userUgi = ugi; this.resource = resource; + this.xtrace_context = XTraceContext.getThreadContext(); } LocalResource getResource() { @@ -169,6 +177,7 @@ private static boolean checkPermissionOfOther(FileSystem fs, Path path, private Path copy(Path sCopy, Path dstdir) throws IOException { + XTraceContext.logEvent(FSDownload.class, "FSDownload", "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 +199,7 @@ private Path copy(Path sCopy, Path dstdir) throws IOException { } private long unpack(File localrsrc, File dst, Pattern pattern) throws IOException { + XTraceContext.logEvent(FSDownload.class, "FSDownload", "Unpacking", "Resource", localrsrc.toString(), "Unpack To", dst.toString()); switch (resource.getType()) { case ARCHIVE: { String lowerDst = dst.getName().toLowerCase(); @@ -262,10 +272,13 @@ private long unpack(File localrsrc, File dst, Pattern pattern) throws IOExceptio @Override public Path call() throws Exception { + XTraceContext.setThreadContext(xtrace_context); + XTraceContext.logEvent(FSDownload.class, "FSDownload", "Localizing resource", "Resource",resource.getResource().toString()); final Path sCopy; try { sCopy = ConverterUtils.getPathFromYarnURL(resource.getResource()); } catch (URISyntaxException e) { + XTraceContext.logEvent(FSDownload.class, "FSDownload", "Invalid resource: "+e.getClass().getName(), "Message", e.getMessage()); throw new IOException("Invalid resource", e); } createDir(destDirPath, cachePerms); @@ -301,7 +314,10 @@ 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())); + XTraceContext.rememberObject(p); + XTraceContext.clearThreadContext(); + 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..9c2455378c0a 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 @@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; public class DefaultContainerExecutor extends ContainerExecutor { @@ -187,20 +188,26 @@ public int launchContainer(Container container, containerIdStr, this.getConf()); LOG.info("launchContainer: " + Arrays.toString(command)); + XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor", "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 { + XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor", "Subprocess finished with exit code "+shExec.getExitCode()); + } else { LOG.info("Container " + containerIdStr + " was marked as inactive. Returning terminated error"); + XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor", + "Container was marked as inactive; returning terminated error"); return ExitCode.TERMINATED.getExitCode(); } } catch (IOException e) { if (null == shExec) { + XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor IOException: " + e.getClass().getName(), + "Message", e.getMessage()); return -1; } int exitCode = shExec.getExitCode(); @@ -221,6 +228,7 @@ public int launchContainer(Container container, container.handle(new ContainerDiagnosticsUpdateEvent(containerId, "Container killed on request. Exit code is " + exitCode)); } + XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor", "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..fc0b74f695d0 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 @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Set; +import java.util.Collection; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; @@ -44,6 +45,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + public class DeletionService extends AbstractService { static final Log LOG = LogFactory.getLog(DeletionService.class); private int debugDelay; @@ -149,6 +153,8 @@ public static class FileDeletionTask implements Runnable { // the dependent task fails then it will be marked as false in // fileDeletionTaskFinished(). private boolean success; + + private Collection xtrace_context; private FileDeletionTask(DeletionService delService, String user, Path subDir, List baseDirs) { @@ -159,6 +165,8 @@ private FileDeletionTask(DeletionService delService, String user, this.successorTaskSet = new HashSet(); this.numberOfPendingPredecessorTasks = new AtomicInteger(0); success = true; + + this.xtrace_context = XTraceContext.getThreadContext(); } /** @@ -204,6 +212,7 @@ public void run() { LOG.debug(this); } boolean error = false; + XTraceContext.joinContext(xtrace_context); if (null == user) { if (baseDirs == null || baseDirs.size() == 0) { LOG.debug("NM deleting absolute path : " + subDir); @@ -246,6 +255,7 @@ public void run() { setSuccess(!error); } fileDeletionTaskFinished(); + XTraceContext.clearThreadContext(); } @Override 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..a7ae13c0b164 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,6 +43,8 @@ import org.apache.hadoop.yarn.server.nodemanager.util.LCEResourcesHandler; import org.apache.hadoop.yarn.util.ConverterUtils; +import edu.berkeley.xtrace.XTraceContext; + public class LinuxContainerExecutor extends ContainerExecutor { private static final Log LOG = LogFactory @@ -143,6 +145,7 @@ public void init() throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("checkLinuxExecutorSetup: " + Arrays.toString(commandArray)); } + XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "init", "args", Arrays.toString(commandArray)); try { shExec.execute(); } catch (ExitCodeException e) { @@ -198,6 +201,7 @@ public void startLocalizer(Path nmPrivateContainerTokensPath, if (LOG.isDebugEnabled()) { LOG.debug("initApplication: " + Arrays.toString(commandArray)); } + XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "Starting localizer", "args", Arrays.toString(commandArray)); try { shExec.execute(); if (LOG.isDebugEnabled()) { @@ -208,6 +212,7 @@ public void startLocalizer(Path nmPrivateContainerTokensPath, LOG.warn("Exit code from container " + locId + " startLocalizer is : " + exitCode, e); logOutput(shExec.getOutput()); + XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "Container finished with exit code "+exitCode); throw new IOException("Application " + appId + " initialization failed" + " (exitCode=" + exitCode + ") with output: " + shExec.getOutput(), e); } @@ -249,6 +254,7 @@ public int launchContainer(Container container, container.getLaunchContext().getEnvironment()); // sanitized env // DEBUG LOG.info("launchContainer: " + Arrays.toString(commandArray)); + XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "Launching container", "args", Arrays.toString(commandArray)); shExec.execute(); if (LOG.isDebugEnabled()) { logOutput(shExec.getOutput()); @@ -263,6 +269,7 @@ public int launchContainer(Container container, } int exitCode = shExec.getExitCode(); LOG.warn("Exit code from container " + containerId + " is : " + exitCode); + XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "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 +311,7 @@ public boolean signalContainer(String user, String pid, Signal signal) if (LOG.isDebugEnabled()) { LOG.debug("signalContainer: " + Arrays.toString(command)); } + XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "signalContainer: Signalling container with signal " + signal.toString(), "args", Arrays.toString(command)); try { shExec.execute(); } catch (ExitCodeException e) { @@ -314,6 +322,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()); + XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "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 +353,7 @@ public void deleteAsUser(String user, Path dir, Path... baseDirs) { if (LOG.isDebugEnabled()) { LOG.debug("deleteAsUser: " + Arrays.toString(commandArray)); } + XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "Delete as user ", "args", Arrays.toString(commandArray)); try { shExec.execute(); if (LOG.isDebugEnabled()) { @@ -354,6 +365,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()); + XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "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..8c3fc5fee1fa 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 @@ -21,6 +21,7 @@ import java.io.IOException; import java.net.ConnectException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -65,6 +66,10 @@ import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + + public class NodeStatusUpdaterImpl extends AbstractService implements NodeStatusUpdater { @@ -346,13 +351,14 @@ public long getRMIdentifier() { } protected void startStatusUpdater() { - + statusUpdaterRunnable = new Runnable() { @Override @SuppressWarnings("unchecked") public void run() { int lastHeartBeatID = 0; while (!isStopped) { + XTraceContext.clearThreadContext(); // 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..c5917abb32cc 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 @@ -17,6 +17,11 @@ */ package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb; +import com.google.protobuf.ByteString; + +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.SerializedException; import org.apache.hadoop.yarn.api.records.URL; @@ -221,4 +226,25 @@ private SerializedExceptionProto convertToProtoFormat(SerializedException t) { return ((SerializedExceptionPBImpl)t).getProto(); } + @Override + public void rememberContext() { + maybeInitBuilder(); + XTraceMetadata ctx = XTraceContext.logMerge(); + if (ctx!=null && ctx.isValid()) { + builder.setXtrace(ByteString.copyFrom(ctx.pack())); + } + } + + @Override + public void joinContext() { + LocalResourceStatusProtoOrBuilder p = viaProto ? proto : builder; + if (p.hasXtrace()) { + ByteString xbs = p.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); + if (xmd.isValid()) { + XTraceContext.joinContext(xmd); + } + } + } + } 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..ff5903f479f4 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,6 +115,8 @@ import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; + public class ContainerManagerImpl extends CompositeService implements ServiceStateChangeListener, ContainerManagementProtocol, EventHandler { @@ -455,6 +457,9 @@ private void startContainerInternal(NMTokenIdentifier nmTokenIdentifier, LOG.info("Start request for " + containerIdStr + " by user " + user); + XTraceContext.logEvent(ContainerManagerImpl.class, "ContainerManagerImpl", "Starting container", + "User", user, "Container ID", containerIdStr); + ContainerLaunchContext launchContext = request.getContainerLaunchContext(); Credentials credentials = parseCredentials(launchContext); @@ -580,6 +585,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 +681,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 +698,7 @@ class ApplicationEventDispatcher implements EventHandler { @Override public void handle(ApplicationEvent event) { + event.joinContext(); Application app = ContainerManagerImpl.this.context.getApplications().get( event.getApplicationID()); @@ -705,6 +714,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 +728,10 @@ public void handle(ContainerManagerEvent event) { case FINISH_CONTAINERS: CMgrCompletedContainersEvent containersFinishedEvent = (CMgrCompletedContainersEvent) event; + XTraceContext.clearThreadContext(); 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..4d76cf10de64 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,6 +18,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.application; +import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; import java.util.Map; @@ -52,6 +53,10 @@ import org.apache.hadoop.yarn.state.StateMachine; import org.apache.hadoop.yarn.state.StateMachineFactory; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + /** * The state machine for the representation of an Application * within the NodeManager. @@ -124,7 +129,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 +304,15 @@ static class AppInitDoneTransition implements @Override public void transition(ApplicationImpl app, ApplicationEvent event) { // Start all the containers waiting for ApplicationInit + Collection start_context = XTraceContext.getThreadContext(); + Collection end_context = new XTraceMetadataCollection(); for (Container container : app.containers.values()) { + XTraceContext.setThreadContext(start_context); app.dispatcher.getEventHandler().handle(new ContainerInitEvent( container.getContainerId())); + end_context = XTraceContext.getThreadContext(end_context); } + XTraceContext.joinContext(end_context); } } @@ -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..5e5404b8a720 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 @@ -70,6 +70,10 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.ConverterUtils; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + public class ContainerImpl implements Container { private final Lock readLock; @@ -99,6 +103,8 @@ public class ContainerImpl implements Container { new ArrayList(); private final List appRsrcs = new ArrayList(); + + private Collection xtrace_localizedresources = new XTraceMetadataCollection(); public ContainerImpl(Configuration conf, Dispatcher dispatcher, ContainerLaunchContext launchContext, Credentials creds, @@ -131,7 +137,7 @@ public ContainerImpl(Configuration conf, Dispatcher dispatcher, private static StateMachineFactory stateMachineFactory = - new StateMachineFactory(ContainerState.NEW) + new StateMachineFactory(ContainerState.NEW, StateMachineFactory.Trace.KEEPALIVE) // From NEW State .addTransition(ContainerState.NEW, EnumSet.of(ContainerState.LOCALIZING, ContainerState.LOCALIZED, @@ -197,19 +203,19 @@ ContainerEventType.KILL_CONTAINER, new KillTransition()) .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_SUCCESS, ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS, - new ExitedWithSuccessTransition(true)) + new ExitedWithSuccessTransition(true), StateMachineFactory.Trace.ALWAYS) .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE, ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, - new ExitedWithFailureTransition(true)) + new ExitedWithFailureTransition(true), StateMachineFactory.Trace.ALWAYS) .addTransition(ContainerState.RUNNING, ContainerState.RUNNING, ContainerEventType.UPDATE_DIAGNOSTICS_MSG, - UPDATE_DIAGNOSTICS_TRANSITION) + UPDATE_DIAGNOSTICS_TRANSITION, StateMachineFactory.Trace.ALWAYS) .addTransition(ContainerState.RUNNING, ContainerState.KILLING, - ContainerEventType.KILL_CONTAINER, new KillTransition()) + ContainerEventType.KILL_CONTAINER, new KillTransition(), StateMachineFactory.Trace.ALWAYS) .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE, ContainerEventType.CONTAINER_KILLED_ON_REQUEST, - new KilledExternallyTransition()) + new KilledExternallyTransition(), StateMachineFactory.Trace.ALWAYS) // From CONTAINER_EXITED_WITH_SUCCESS State .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE, @@ -599,9 +605,11 @@ public ContainerState transition(ContainerImpl container, return ContainerState.LOCALIZING; } container.localizedResources.put(rsrcEvent.getLocation(), syms); + container.xtrace_localizedresources = XTraceContext.getThreadContext(container.xtrace_localizedresources); if (!container.pendingResources.isEmpty()) { return ContainerState.LOCALIZING; } + XTraceContext.setThreadContext(container.xtrace_localizedresources); container.dispatcher.getEventHandler().handle( new ContainersLauncherEvent(container, ContainersLauncherEventType.LAUNCH_CONTAINER)); @@ -838,6 +846,7 @@ public void transition(ContainerImpl container, ContainerEvent event) { @Override public void handle(ContainerEvent event) { + event.joinContext(); try { 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/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java index 3f54ea4ba43a..7dd5facd3eff 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java @@ -27,6 +27,7 @@ import java.io.OutputStream; import java.io.PrintStream; import java.util.ArrayList; +import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; import java.util.List; @@ -71,6 +72,9 @@ import org.apache.hadoop.yarn.util.Apps; import org.apache.hadoop.yarn.util.ConverterUtils; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + public class ContainerLaunch implements Callable { private static final Log LOG = LogFactory.getLog(ContainerLaunch.class); @@ -98,6 +102,8 @@ public class ContainerLaunch implements Callable { private final LocalDirsHandlerService dirsHandler; + private Collection xtrace; + public ContainerLaunch(Context context, Configuration configuration, Dispatcher dispatcher, ContainerExecutor exec, Application app, Container container, LocalDirsHandlerService dirsHandler) { @@ -114,11 +120,15 @@ public ContainerLaunch(Context context, Configuration configuration, this.maxKillWaitTime = conf.getLong(YarnConfiguration.NM_PROCESS_KILL_WAIT_MS, YarnConfiguration.DEFAULT_NM_PROCESS_KILL_WAIT_MS); + this.xtrace = XTraceContext.getThreadContext(); } - + @Override @SuppressWarnings("unchecked") // dispatcher not typed public Integer call() { + + XTraceContext.setThreadContext(this.xtrace); + final ContainerLaunchContext launchContext = container.getLaunchContext(); final Map> localResources = container.getLocalizedResources(); 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/launcher/ContainersLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java index 4e374fd5b54d..d116bfec7a30 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java @@ -45,6 +45,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; + /** * The launcher for the containers. This service should be started only after * the {@link ResourceLocalizationService} is started as it depends on creation @@ -109,11 +111,13 @@ protected void serviceStop() throws Exception { @Override public void handle(ContainersLauncherEvent event) { + event.joinContext(); // TODO: ContainersLauncher launches containers one by one!! Container container = event.getContainer(); ContainerId containerId = container.getContainerId(); switch (event.getType()) { case LAUNCH_CONTAINER: + XTraceContext.logEvent(ContainersLauncher.class, "ContainersLauncher", "Launching container"); Application app = context.getApplications().get( containerId.getApplicationAttemptId().getApplicationId()); @@ -126,6 +130,7 @@ public void handle(ContainersLauncherEvent event) { launch)); break; case CLEANUP_CONTAINER: + XTraceContext.logEvent(ContainersLauncher.class, "ContainersLauncher", "Cleanup container"); RunningContainer rContainerDatum = running.remove(containerId); if (rContainerDatum == null) { // Container not launched. So nothing needs to be done. 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/localizer/ContainerLocalizer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java index 66f21f6a5714..6ef0278d058d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java @@ -24,6 +24,7 @@ import java.security.PrivilegedAction; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -71,6 +72,9 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + public class ContainerLocalizer { static final Log LOG = LogFactory.getLog(ContainerLocalizer.class); @@ -225,6 +229,7 @@ protected void localizeFiles(LocalizationProtocol nodemanager, CompletionService cs, UserGroupInformation ugi) throws IOException { while (true) { + XTraceContext.clearThreadContext(); try { LocalizerStatus status = createStatus(); LocalizerHeartbeatResponse response = nodemanager.heartbeat(status); @@ -232,11 +237,13 @@ protected void localizeFiles(LocalizationProtocol nodemanager, case LIVE: List newRsrcs = response.getResourceSpecs(); for (ResourceLocalizationSpec newRsrc : newRsrcs) { + newRsrc.getResource().joinContext(); if (!pendingResources.containsKey(newRsrc.getResource())) { pendingResources.put(newRsrc.getResource(), cs.submit(download( new Path(newRsrc.getDestinationDirectory().getFile()), newRsrc.getResource(), ugi))); } + XTraceContext.clearThreadContext(); } break; case DIE: @@ -282,6 +289,8 @@ private LocalizerStatus createStatus() throws InterruptedException { if (fPath.isDone()) { try { Path localPath = fPath.get(); + XTraceContext.joinObject(localPath); + stat.rememberContext(); stat.setLocalPath( ConverterUtils.getYarnUrlFromPath(localPath)); stat.setLocalSize( @@ -300,6 +309,7 @@ private LocalizerStatus createStatus() throws InterruptedException { stat.setStatus(ResourceStatusType.FETCH_PENDING); } currentResources.add(stat); + XTraceContext.clearThreadContext(); } LocalizerStatus status = recordFactory.newRecordInstance(LocalizerStatus.class); @@ -317,6 +327,7 @@ public static void main(String[] argv) throws Throwable { // MKDIR $x/$user/appcache/$appid/filecache // LOAD $x/$user/appcache/$appid/appTokens try { + XTraceContext.startTrace("ContainerLocalizer", "ContainerLocalizer launched..."); String user = argv[0]; String appId = argv[1]; String locId = argv[2]; @@ -339,7 +350,9 @@ public static void main(String[] argv) throws Throwable { new ContainerLocalizer(FileContext.getLocalFSFileContext(), user, appId, locId, localDirs, RecordFactoryProvider.getRecordFactory(null)); - System.exit(localizer.runLocalization(nmAddr)); + int retCode = localizer.runLocalization(nmAddr); + XTraceContext.joinParentProcess(); + System.exit(retCode); } catch (Throwable e) { // Print error to stdout so that LCE can use it. e.printStackTrace(System.out); 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/localizer/LocalResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java index 70bead7320a4..72140f9bed6e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java @@ -191,4 +191,12 @@ public String toString() { sb.append(getPattern()).append(" }"); return sb.toString(); } + + @Override + public void rememberContext() { + } + + @Override + 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/containermanager/localizer/LocalResourcesTrackerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java index dfbeb449349f..1eb1406d929b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java @@ -107,6 +107,7 @@ public LocalResourcesTrackerImpl(String user, Dispatcher dispatcher, */ @Override public synchronized void handle(ResourceEvent event) { + event.joinContext(); LocalResourceRequest req = event.getLocalResourceRequest(); LocalizedResource rsrc = localrsrc.get(req); switch (event.getType()) { @@ -317,4 +318,4 @@ public long nextUniqueNumber() { public LocalizedResource getLocalizedResource(LocalResourceRequest request) { return localrsrc.get(request); } -} \ No newline at end of file +} 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/localizer/LocalizedResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java index f49e942ffd21..67f5d2114cfc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java @@ -71,7 +71,7 @@ public class LocalizedResource implements EventHandler { private static final StateMachineFactory stateMachineFactory = new StateMachineFactory(ResourceState.INIT) + ResourceEventType,ResourceEvent>(ResourceState.INIT, StateMachineFactory.Trace.KEEPALIVE) // From INIT (ref == 0, awaiting req) .addTransition(ResourceState.INIT, ResourceState.DOWNLOADING, @@ -81,7 +81,7 @@ ResourceEventType.REQUEST, new FetchResourceTransition()) .addTransition(ResourceState.DOWNLOADING, ResourceState.DOWNLOADING, ResourceEventType.REQUEST, new FetchResourceTransition()) // TODO: Duplicate addition!! .addTransition(ResourceState.DOWNLOADING, ResourceState.LOCALIZED, - ResourceEventType.LOCALIZED, new FetchSuccessTransition()) + ResourceEventType.LOCALIZED, new FetchSuccessTransition(), StateMachineFactory.Trace.ALWAYS) .addTransition(ResourceState.DOWNLOADING,ResourceState.DOWNLOADING, ResourceEventType.RELEASE, new ReleaseTransition()) .addTransition(ResourceState.DOWNLOADING, ResourceState.FAILED, @@ -180,6 +180,7 @@ public void unlock() { @Override public void handle(ResourceEvent event) { try { + event.joinContext(); this.writeLock.lock(); Path resourcePath = event.getLocalResourceRequest().getPath(); 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/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java index 1fdb0821708b..2b92348ff87d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java @@ -119,6 +119,10 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + public class ResourceLocalizationService extends CompositeService implements EventHandler, LocalizationProtocol { @@ -300,6 +304,7 @@ public void serviceStop() throws Exception { @Override public void handle(LocalizationEvent event) { + event.joinContext(); // TODO: create log dir as $logdir/$user/$appId switch (event.getType()) { case INIT_APPLICATION_RESOURCES: @@ -364,6 +369,8 @@ private void handleInitContainerResources( c.getUser(), c.getContainerId(), c.getCredentials()); Map> rsrcs = rsrcReqs.getRequestedResources(); + Collection startCtx = XTraceContext.getThreadContext(); + Collection endCtxs = new XTraceMetadataCollection(); for (Map.Entry> e : rsrcs.entrySet()) { LocalResourcesTracker tracker = @@ -371,9 +378,12 @@ private void handleInitContainerResources( c.getContainerId().getApplicationAttemptId() .getApplicationId()); for (LocalResourceRequest req : e.getValue()) { + XTraceContext.setThreadContext(startCtx); tracker.handle(new ResourceRequestEvent(req, e.getKey(), ctxt)); + endCtxs = XTraceContext.getThreadContext(endCtxs); } } + XTraceContext.joinContext(endCtxs); } private void handleCacheCleanup(LocalizationEvent event) { @@ -392,19 +402,25 @@ private void handleCacheCleanup(LocalizationEvent event) { @SuppressWarnings("unchecked") private void handleCleanupContainerResources( ContainerLocalizationCleanupEvent rsrcCleanup) { + rsrcCleanup.joinContext(); Container c = rsrcCleanup.getContainer(); Map> rsrcs = rsrcCleanup.getResources(); + Collection start_context = XTraceContext.getThreadContext(); + Collection end_context = new XTraceMetadataCollection(); for (Map.Entry> e : rsrcs.entrySet()) { LocalResourcesTracker tracker = getLocalResourcesTracker(e.getKey(), c.getUser(), c.getContainerId().getApplicationAttemptId() .getApplicationId()); for (LocalResourceRequest req : e.getValue()) { + XTraceContext.setThreadContext(start_context); tracker.handle(new ResourceReleaseEvent(req, c.getContainerId())); + end_context = XTraceContext.getThreadContext(end_context); } } + XTraceContext.setThreadContext(end_context); String locId = ConverterUtils.toString(c.getContainerId()); localizerTracker.cleanupPrivLocalizers(locId); @@ -571,6 +587,7 @@ public void serviceStop() throws Exception { @Override public void handle(LocalizerEvent event) { + event.joinContext(); String locId = event.getLocalizerId(); switch (event.getType()) { case REQUEST_RESOURCE_LOCALIZATION: @@ -700,11 +717,13 @@ public void run() { try { // TODO shutdown, better error handling esp. DU while (!Thread.currentThread().isInterrupted()) { + XTraceContext.clearThreadContext(); try { Future completed = queue.take(); LocalizerResourceRequestEvent assoc = pending.remove(completed); try { Path local = completed.get(); + XTraceContext.joinObject(local); if (null == assoc) { LOG.error("Localized unkonwn resource to " + completed); // TODO delete @@ -775,6 +794,7 @@ public void addResource(LocalizerResourceRequestEvent request) { */ private LocalResource findNextResource() { // TODO: Synchronization + XTraceContext.clearThreadContext(); for (Iterator i = pending.iterator(); i.hasNext();) { LocalizerResourceRequestEvent evt = i.next(); @@ -793,6 +813,7 @@ private LocalResource findNextResource() { */ if (nRsrc.tryAcquire()) { if (nRsrc.getState().equals(ResourceState.DOWNLOADING)) { + evt.joinContext(); LocalResourceRequest nextRsrc = nRsrc.getRequest(); LocalResource next = recordFactory.newRecordInstance(LocalResource.class); @@ -803,6 +824,7 @@ private LocalResource findNextResource() { next.setVisibility(evt.getVisibility()); next.setPattern(evt.getPattern()); scheduled.put(nextRsrc, evt); + XTraceContext.clearThreadContext(); return next; } else { // Need to release acquired lock @@ -857,6 +879,7 @@ LocalizerHeartbeatResponse update( */ for (LocalResourceStatus stat : remoteResourceStatuses) { + stat.joinContext(); LocalResource rsrc = stat.getResource(); LocalResourceRequest req = null; try { @@ -870,6 +893,7 @@ LocalizerHeartbeatResponse update( LOG.error("Unknown resource reported: " + req); continue; } + assoc.joinContext(); switch (stat.getStatus()) { case FETCH_SUCCESS: // notify resource @@ -931,6 +955,7 @@ LocalizerHeartbeatResponse update( req, stat.getException().getMessage())); break; } + XTraceContext.clearThreadContext(); } response.setResourceSpecs(rsrcs); return response; @@ -1038,15 +1063,19 @@ private void writeCredentials(Path nmPrivateCTokensPath) static class CacheCleanup extends Thread { private final Dispatcher dispatcher; + private Collection xtrace_context; public CacheCleanup(Dispatcher dispatcher) { super("CacheCleanup"); this.dispatcher = dispatcher; + this.xtrace_context = XTraceContext.getThreadContext(); } @Override @SuppressWarnings("unchecked") // dispatcher not typed public void run() { + XTraceContext.setThreadContext(xtrace_context); + XTraceContext.logEvent(CacheCleanup.class, "Cache Cleanup Thread", "Cache Cleanup Thread started"); dispatcher.getEventHandler().handle( new LocalizationEvent(LocalizationEventType.CACHE_CLEANUP)); } 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/logaggregation/LogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java index efe8984694e4..7f7dda2bef16 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java @@ -406,6 +406,7 @@ private void stopApp(ApplicationId appId) { @Override public void handle(LogHandlerEvent event) { + event.joinContext(); switch (event.getType()) { case APPLICATION_STARTED: LogHandlerAppStartedEvent appStartEvent = 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/loghandler/NonAggregatingLogHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java index 06b654ae0640..62cce082d0e4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -42,6 +43,9 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * Log Handler which schedules deletion of log files based on the configured log * retention time. @@ -98,6 +102,7 @@ protected void serviceStop() throws Exception { @SuppressWarnings("unchecked") @Override public void handle(LogHandlerEvent event) { + event.joinContext(); switch (event.getType()) { case APPLICATION_STARTED: LogHandlerAppStartedEvent appStartedEvent = @@ -142,15 +147,18 @@ ScheduledThreadPoolExecutor createScheduledThreadPoolExecutor( class LogDeleterRunnable implements Runnable { private String user; private ApplicationId applicationId; + private Collection xtrace_context; public LogDeleterRunnable(String user, ApplicationId applicationId) { this.user = user; this.applicationId = applicationId; + this.xtrace_context = XTraceContext.getThreadContext(); } @Override @SuppressWarnings("unchecked") public void run() { + XTraceContext.setThreadContext(xtrace_context); List rootLogDirs = NonAggregatingLogHandler.this.dirsHandler.getLogDirs(); Path[] localAppLogDirs = new Path[rootLogDirs.size()]; @@ -166,6 +174,7 @@ public void run() { ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED)); NonAggregatingLogHandler.this.delService.delete(user, null, localAppLogDirs); + XTraceContext.clearThreadContext(); } @Override 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/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java index b681b34aad5e..8ef9bf92be68 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -42,6 +43,9 @@ import com.google.common.base.Preconditions; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + public class ContainersMonitorImpl extends AbstractService implements ContainersMonitor { @@ -308,13 +312,17 @@ boolean isProcessTreeOverLimit(ResourceCalculatorProcessTree pTree, } private class MonitoringThread extends Thread { + private Collection xtrace_context; + public MonitoringThread() { super("Container Monitor"); + this.xtrace_context = XTraceContext.getThreadContext(); } @Override public void run() { - + XTraceContext.joinContext(xtrace_context); + XTraceContext.logEvent(MonitoringThread.class, "Container Monitor Thread", "Container Monitor Thread started"); while (true) { // Print the processTrees for debugging. @@ -525,6 +533,7 @@ public boolean isVmemCheckEnabled() { @Override public void handle(ContainersMonitorEvent monitoringEvent) { + monitoringEvent.joinContext(); if (!isEnabled()) { return; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto index 166ea5b37026..0dd66a1abdb7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto @@ -35,6 +35,7 @@ message LocalResourceStatusProto { optional URLProto localPath = 3; optional int64 localSize = 4; optional SerializedExceptionProto exception = 5; + optional bytes xtrace = 6; // X-Trace metadata } message LocalizerStatusProto { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java index 1eeab7c81ae0..c2c7c886a762 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java @@ -79,4 +79,9 @@ && getLocalPath().equals(other.getLocalPath())) public int hashCode() { return 4344; } + @Override + public void rememberContext(){} + @Override + public void joinContext(){}; } + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java index 14456f535a9a..ff9c72bcccfb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java @@ -84,6 +84,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider; import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import edu.berkeley.xtrace.XTraceContext; + @SuppressWarnings("unchecked") @Private public class ApplicationMasterService extends AbstractService implements @@ -530,6 +532,7 @@ public void registerAppAttempt(ApplicationAttemptId attemptId) { // attemptID get registered response.setResponseId(-1); LOG.info("Registering app attempt : " + attemptId); + XTraceContext.logEvent("ApplicationMasterService", "Registering application attempt", "Attempt ID", attemptId); responseMap.put(attemptId, response); rmContext.getNMTokenSecretManager().registerApplicationAttempt(attemptId); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index 1f7a8477d6ee..e92532dc6792 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -94,6 +94,7 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Records; +import edu.berkeley.xtrace.XTraceContext; /** @@ -201,6 +202,7 @@ ApplicationId getNewApplicationId() { .newApplicationId(recordFactory, ResourceManager.clusterTimeStamp, applicationCounter.incrementAndGet()); LOG.info("Allocated new applicationId: " + applicationId.getId()); + XTraceContext.logEvent(ClientRMService.class, "ClientRMService", "New application ID allocated", "Application ID", applicationId.getId()); return applicationId; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java index e9a043693e10..08b2420fea9f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java @@ -142,6 +142,7 @@ public int getUnusableNodes(Collection unUsableNodes) { @Override public void handle(NodesListManagerEvent event) { + event.joinContext(); RMNode eventNode = event.getNode(); switch (event.getType()) { case NODE_UNUSABLE: diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index 6439df1c2251..da9d32cae403 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -54,6 +54,8 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import edu.berkeley.xtrace.XTraceContext; + /** * This class manages the list of applications for the resource manager. */ @@ -241,6 +243,7 @@ protected void submitApplication( ApplicationSubmissionContext submissionContext, long submitTime, boolean isRecovered, String user) throws YarnException { ApplicationId applicationId = submissionContext.getApplicationId(); + XTraceContext.logEvent("RMAppManager", "Submitting application", "Application ID", applicationId.getId()); // Validation of the ApplicationSubmissionContext needs to be completed // here. Only those fields that are dependent on RM's configuration are @@ -383,6 +386,7 @@ public void recover(RMState state) throws Exception { @Override public void handle(RMAppManagerEvent event) { + event.joinContext(); ApplicationId applicationId = event.getApplicationId(); LOG.debug("RMAppManager processing event for " + applicationId + " of type " + event.getType()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index 841f387e7d9a..fcd199ea9fbe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -90,6 +90,7 @@ import org.apache.hadoop.yarn.webapp.WebApps.Builder; import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; /** * The ResourceManager is the main class that is a set of components. @@ -401,12 +402,14 @@ public void run() { SchedulerEvent event; while (!stopped && !Thread.currentThread().isInterrupted()) { + XTraceContext.clearThreadContext(); try { event = eventQueue.take(); } catch (InterruptedException e) { LOG.error("Returning, interrupted : " + e); return; // TODO: Kill RM. } + event.joinContext(); try { scheduler.handle(event); @@ -444,6 +447,7 @@ protected void serviceStop() throws Exception { @Override public void handle(SchedulerEvent event) { + event.joinContext(); try { int qSize = eventQueue.size(); if (qSize !=0 && qSize %1000 == 0) { @@ -454,6 +458,7 @@ public void handle(SchedulerEvent event) { LOG.info("Very low remaining capacity on scheduler event queue: " + remCapacity); } + event.rememberContext(); this.eventQueue.put(event); } catch (InterruptedException e) { throw new YarnRuntimeException(e); @@ -473,6 +478,7 @@ public ApplicationEventDispatcher(RMContext rmContext) { @Override public void handle(RMAppEvent event) { + event.joinContext(); ApplicationId appID = event.getApplicationId(); RMApp rmApp = this.rmContext.getRMApps().get(appID); if (rmApp != null) { @@ -528,6 +534,7 @@ public ApplicationAttemptEventDispatcher(RMContext rmContext) { @Override public void handle(RMAppAttemptEvent event) { + event.joinContext(); ApplicationAttemptId appAttemptID = event.getApplicationAttemptId(); ApplicationId appAttemptId = appAttemptID.getApplicationId(); RMApp rmApp = this.rmContext.getRMApps().get(appAttemptId); @@ -557,6 +564,7 @@ public NodeEventDispatcher(RMContext rmContext) { @Override public void handle(RMNodeEvent event) { + event.joinContext(); NodeId nodeId = event.getNodeId(); RMNode node = this.rmContext.getRMNodes().get(nodeId); if (node != null) { @@ -815,6 +823,7 @@ public void recover(RMState state) throws Exception { } public static void main(String argv[]) { + System.out.println("I am ResourceManager, king of kings. Look on my works ye mighty, and despair."); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); StringUtils.startupShutdownMessage(ResourceManager.class, argv, LOG); try { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java index a1c1a40d1bfa..cc2088481dc5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java @@ -24,6 +24,7 @@ import java.security.PrivilegedAction; import java.util.ArrayList; import java.util.List; +import java.util.Collection; import java.util.Map; import org.apache.commons.logging.Log; @@ -63,6 +64,9 @@ import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * The launch of the AM itself. */ @@ -80,6 +84,8 @@ public class AMLauncher implements Runnable { @SuppressWarnings("rawtypes") private final EventHandler handler; + + private Collection xtrace_context; public AMLauncher(RMContext rmContext, RMAppAttempt application, AMLauncherEventType eventType, Configuration conf) { @@ -89,6 +95,7 @@ public AMLauncher(RMContext rmContext, RMAppAttempt application, this.rmContext = rmContext; this.handler = rmContext.getDispatcher().getEventHandler(); this.masterContainer = application.getMasterContainer(); + this.xtrace_context = XTraceContext.getThreadContext(); } private void connect() throws IOException { @@ -103,7 +110,9 @@ private void launch() throws IOException, YarnException { ApplicationSubmissionContext applicationContext = application.getSubmissionContext(); LOG.info("Setting up container " + masterContainer - + " for AM " + application.getAppAttemptId()); + + " for AM " + application.getAppAttemptId()); + XTraceContext.logEvent(AMLauncher.class, "AMLauncher", "Setting up container for application master", + "Container", application.getMasterContainer(), "Application Master ID", application.getAppAttemptId()); ContainerLaunchContext launchContext = createAMContainerLaunchContext(applicationContext, masterContainerID); @@ -242,10 +251,14 @@ protected Token getAMRMToken() { @SuppressWarnings("unchecked") public void run() { + XTraceContext.setThreadContext(this.xtrace_context); + switch (eventType) { case LAUNCH: try { - LOG.info("Launching master" + application.getAppAttemptId()); + LOG.info("Launching master " + application.getAppAttemptId()); + XTraceContext.logEvent(AMLauncher.class, "AMLauncher", "Launching application master", + "Application Master ID", application.getAppAttemptId()); launch(); handler.handle(new RMAppAttemptEvent(application.getAppAttemptId(), RMAppAttemptEventType.LAUNCHED)); @@ -260,6 +273,8 @@ public void run() { case CLEANUP: try { LOG.info("Cleaning master " + application.getAppAttemptId()); + XTraceContext.logEvent(AMLauncher.class, "AMLauncher", "Cleaning application master", + "Application Master ID", application.getAppAttemptId()); cleanup(); } catch(IOException ie) { LOG.info("Error cleaning master ", ie); @@ -277,6 +292,8 @@ public void run() { LOG.warn("Received unknown event-type " + eventType + ". Ignoring."); break; } + + XTraceContext.clearThreadContext(); } private void parseAndThrowException(Throwable t) throws YarnException, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java index af02b198cf76..8dfa5e3ebc05 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java @@ -109,6 +109,7 @@ private void cleanup(RMAppAttempt application) { @Override public synchronized void handle(AMLauncherEvent appEvent) { + appEvent.joinContext(); AMLauncherEventType event = appEvent.getType(); RMAppAttempt application = appEvent.getAppAttempt(); switch (event) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java index a1c234cad236..7474f603d9d9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java @@ -21,9 +21,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.event.AbstractEvent; +import edu.berkeley.xtrace.XTraceMetadata; + public class RMAppEvent extends AbstractEvent{ private final ApplicationId appId; + public XTraceMetadata last_transition; public RMAppEvent(ApplicationId appId, RMAppEventType type) { super(type); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java index a11b05eedc97..1abde9dafbf3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java @@ -118,7 +118,7 @@ public class RMAppImpl implements RMApp, Recoverable { = new StateMachineFactory(RMAppState.NEW) + RMAppEvent>(RMAppState.NEW, StateMachineFactory.Trace.KEEPALIVE) // Transitions from NEW state @@ -169,15 +169,15 @@ RMAppEventType.KILL, new KillAppAndAttemptTransition()) .addTransition(RMAppState.RUNNING, RMAppState.RUNNING, RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition()) .addTransition(RMAppState.RUNNING, RMAppState.FINISHING, - RMAppEventType.ATTEMPT_FINISHING, new RMAppFinishingTransition()) + RMAppEventType.ATTEMPT_FINISHING, new RMAppFinishingTransition(), StateMachineFactory.Trace.ALWAYS) .addTransition(RMAppState.RUNNING, RMAppState.FINISHED, - RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION) + RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION, StateMachineFactory.Trace.ALWAYS) .addTransition(RMAppState.RUNNING, EnumSet.of(RMAppState.SUBMITTED, RMAppState.FAILED), RMAppEventType.ATTEMPT_FAILED, - new AttemptFailedTransition(RMAppState.SUBMITTED)) + new AttemptFailedTransition(RMAppState.SUBMITTED), StateMachineFactory.Trace.ALWAYS) .addTransition(RMAppState.RUNNING, RMAppState.KILLED, - RMAppEventType.KILL, new KillAppAndAttemptTransition()) + RMAppEventType.KILL, new KillAppAndAttemptTransition(), StateMachineFactory.Trace.ALWAYS) // Transitions from FINISHING state .addTransition(RMAppState.FINISHING, RMAppState.FINISHED, @@ -568,6 +568,7 @@ public int getMaxAppAttempts() { @Override public void handle(RMAppEvent event) { + event.joinContext(); this.writeLock.lock(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java index 4a877cadae78..18fdfbd38aeb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java @@ -99,6 +99,8 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.resource.Resources; +import edu.berkeley.xtrace.XTraceContext; + @SuppressWarnings({"unchecked", "rawtypes"}) public class RMAppAttemptImpl implements RMAppAttempt, Recoverable { @@ -162,7 +164,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable { stateMachineFactory = new StateMachineFactory(RMAppAttemptState.NEW) + RMAppAttemptEvent>(RMAppAttemptState.NEW, StateMachineFactory.Trace.NEVER) // Transitions from NEW State .addTransition(RMAppAttemptState.NEW, RMAppAttemptState.SUBMITTED, @@ -587,6 +589,7 @@ private void setMasterContainer(Container container) { @Override public void handle(RMAppAttemptEvent event) { + event.joinContext(); this.writeLock.lock(); @@ -786,6 +789,8 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt, BuilderUtils.newResourceRequest( AM_CONTAINER_PRIORITY, ResourceRequest.ANY, appAttempt .getSubmissionContext().getResource(), 1); + XTraceContext.logEvent(RMAppAttemptImpl.class, "RMAppAttemptImpl", "Requesting container for ApplicationMaster"); + request.rememberContext(); // SchedulerUtils.validateResourceRequests is not necessary because // AM resource has been checked when submission diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java index d44fd3f32c2e..f92cd6e6250d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java @@ -50,7 +50,7 @@ public class RMContainerImpl implements RMContainer { RMContainerEventType, RMContainerEvent> stateMachineFactory = new StateMachineFactory( - RMContainerState.NEW) + RMContainerState.NEW, StateMachineFactory.Trace.KEEPALIVE) // Transitions from NEW state .addTransition(RMContainerState.NEW, RMContainerState.ALLOCATED, @@ -205,6 +205,8 @@ public String toString() { @Override public void handle(RMContainerEvent event) { + // X-Trace: this handle method is only ever called synchronously (it seems) + event.joinContext(); LOG.debug("Processing " + event.getContainerId() + " of type " + event.getType()); try { writeLock.lock(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java index 70507f6ad2b0..e00690a07cfa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java @@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; /** * This class is used to keep track of all the applications/containers @@ -118,7 +119,7 @@ public class RMNodeImpl implements RMNode, EventHandler { = new StateMachineFactory(NodeState.NEW) + RMNodeEvent>(NodeState.NEW, StateMachineFactory.Trace.NEVER) //Transitions from NEW state .addTransition(NodeState.NEW, NodeState.RUNNING, @@ -350,6 +351,7 @@ public NodeHeartbeatResponse getLastNodeHeartBeatResponse() { } public void handle(RMNodeEvent event) { + event.joinContext(); LOG.debug("Processing " + event.getNodeId() + " of type " + event.getType()); try { writeLock.lock(); @@ -484,8 +486,9 @@ public static class CleanUpContainerTransition implements @Override public void transition(RMNodeImpl rmNode, RMNodeEvent event) { - rmNode.containersToClean.add((( - RMNodeCleanContainerEvent) event).getContainerId()); + ContainerId id = ((RMNodeCleanContainerEvent) event).getContainerId(); + id.rememberContext(); + rmNode.containersToClean.add(id); } } @@ -557,12 +560,17 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { for (ContainerStatus remoteContainer : statusEvent.getContainers()) { ContainerId containerId = remoteContainer.getContainerId(); + XTraceContext.clearThreadContext(); + containerId.joinContext(); + // Don't bother with containers already scheduled for cleanup, or for // applications already killed. The scheduler doens't need to know any // more about this container if (rmNode.containersToClean.contains(containerId)) { LOG.info("Container " + containerId + " already scheduled for " + "cleanup, no further processing"); + XTraceContext.logEvent(RMNodeImpl.class, "Node cleanup", "Container already scheduled for " + + "cleanup, no further processing", "Container ID", containerId); continue; } if (rmNode.finishedApplications.contains(containerId @@ -570,6 +578,8 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { LOG.info("Container " + containerId + " belongs to an application that is already killed," + " no further processing"); + XTraceContext.logEvent(RMNodeImpl.class, "Node cleanup", "Container belongs to an application that is already killed" + +", no further processing", "Container ID", containerId); continue; } @@ -580,6 +590,7 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { rmNode.justLaunchedContainers.put(containerId, remoteContainer); newlyLaunchedContainers.add(remoteContainer); } + rmNode.justLaunchedContainers.get(containerId).getContainerId().rememberContext(); } else { // A finished container rmNode.justLaunchedContainers.remove(containerId); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index 29c4d4b9de31..7ef18c2694e7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -80,6 +80,8 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import edu.berkeley.xtrace.XTraceContext; + @LimitedPrivate("yarn") @Evolving @SuppressWarnings("unchecked") @@ -637,9 +639,13 @@ private synchronized void nodeUpdate(RMNode nm) { // Process completed containers for (ContainerStatus completedContainer : completedContainers) { ContainerId containerId = completedContainer.getContainerId(); + XTraceContext.clearThreadContext(); + containerId.joinContext(); LOG.debug("Container FINISHED: " + containerId); + XTraceContext.logEvent(CapacityScheduler.class, "Container Finished", "Container Finished", "Container ID",containerId); completedContainer(getRMContainer(containerId), completedContainer, RMContainerEventType.FINISHED); + XTraceContext.clearThreadContext(); } // Now node data structures are upto date and ready for scheduling. @@ -691,6 +697,9 @@ private synchronized void nodeUpdate(RMNode nm) { } private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { + XTraceContext.clearThreadContext(); + containerId.joinContext(); + // Get the application for the finished container ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); FiCaSchedulerApp application = getApplication(applicationAttemptId); @@ -704,10 +713,12 @@ private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode } application.containerLaunchedOnNode(containerId, node.getNodeID()); + XTraceContext.clearThreadContext(); } @Override public void handle(SchedulerEvent event) { + event.joinContext(); switch(event.getType()) { case NODE_ADDED: { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index c2c5d27576dd..8f6c0b7ad95d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -67,6 +67,8 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import edu.berkeley.xtrace.XTraceContext; + @Private @Unstable public class LeafQueue implements CSQueue { @@ -1301,6 +1303,12 @@ Token createContainerToken( private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, ResourceRequest request, NodeType type, RMContainer rmContainer) { + + XTraceContext.clearThreadContext(); + request.joinContext(); + + try { // xtrace try + if (LOG.isDebugEnabled()) { LOG.debug("assignContainers: node=" + node.getHostName() + " application=" + application.getApplicationId().getId() @@ -1382,6 +1390,10 @@ private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode nod return request.getCapability(); } + + } finally { // xtrace finally + XTraceContext.clearThreadContext(); + } } private void reserve(FiCaSchedulerApp application, Priority priority, @@ -1617,4 +1629,4 @@ public Resource getTotalResourcePending() { return ret; } -} +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index a261dbfd5a27..d29c6eeaa29f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -66,6 +66,9 @@ import com.google.common.collect.HashMultiset; import com.google.common.collect.Multiset; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * Represents an Application from the viewpoint of the scheduler. * Each running Application in the RM corresponds to one instance @@ -308,10 +311,13 @@ synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node, synchronized public List pullNewlyAllocatedContainers() { List returnContainerList = new ArrayList( newlyAllocatedContainers.size()); + Collection start_context = XTraceContext.getThreadContext(); for (RMContainer rmContainer : newlyAllocatedContainers) { rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(), RMContainerEventType.ACQUIRED)); returnContainerList.add(rmContainer.getContainer()); + rmContainer.getContainerId().rememberContext(); + XTraceContext.setThreadContext(start_context); } newlyAllocatedContainers.clear(); return returnContainerList; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java index 81fbc16f16a7..6929e904c78c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java @@ -74,6 +74,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent; @@ -91,6 +92,8 @@ import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; + /** * A scheduler that schedules resources between a set of queues. The scheduler * keeps track of the resources used by each queue, and attempts to maintain @@ -852,6 +855,9 @@ public Allocation allocate(ApplicationAttemptId appAttemptId, * Process a container which has launched on a node, as reported by the node. */ private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) { + XTraceContext.clearThreadContext(); + containerId.joinContext(); + // Get the application for the finished container ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); FSSchedulerApp application = applications.get(applicationAttemptId); @@ -863,6 +869,7 @@ private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode no } application.containerLaunchedOnNode(containerId, node.getNodeID()); + XTraceContext.clearThreadContext(); } /** @@ -890,9 +897,13 @@ private synchronized void nodeUpdate(RMNode nm) { // Process completed containers for (ContainerStatus completedContainer : completedContainers) { ContainerId containerId = completedContainer.getContainerId(); + XTraceContext.clearThreadContext(); + containerId.joinContext(); LOG.debug("Container FINISHED: " + containerId); + XTraceContext.logEvent(CapacityScheduler.class, "Container Finished", "Container Finished", "Container ID", containerId); completedContainer(getRMContainer(containerId), completedContainer, RMContainerEventType.FINISHED); + XTraceContext.clearThreadContext(); } // Assign new containers... diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index d971f3b4496e..4cc55b9240b9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -74,6 +75,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerUtils; @@ -90,6 +92,10 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + @LimitedPrivate("yarn") @Evolving @SuppressWarnings("unchecked") @@ -258,7 +264,10 @@ public Allocation allocate( clusterResource, minimumAllocation, maximumAllocation); // Release containers + Collection start_context = XTraceContext.getThreadContext(); for (ContainerId releasedContainer : release) { + XTraceContext.setThreadContext(start_context); + releasedContainer.joinContext(); RMContainer rmContainer = getRMContainer(releasedContainer); if (rmContainer == null) { RMAuditLogger.logFailure(application.getUser(), @@ -273,6 +282,7 @@ public Allocation allocate( SchedulerUtils.RELEASED_CONTAINER), RMContainerEventType.RELEASED); } + XTraceContext.setThreadContext(start_context); synchronized (application) { @@ -553,6 +563,9 @@ private int assignOffSwitchContainers(FiCaSchedulerNode node, private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, int assignableContainers, ResourceRequest request, NodeType type) { + XTraceContext.clearThreadContext(); + request.joinContext(); + LOG.debug("assignContainers:" + " node=" + node.getRMNode().getNodeAddress() + " application=" + application.getApplicationId().getId() + @@ -607,6 +620,7 @@ private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application } } + XTraceContext.clearThreadContext(); return assignedContainers; } @@ -629,9 +643,13 @@ private synchronized void nodeUpdate(RMNode rmNode) { // Process completed containers for (ContainerStatus completedContainer : completedContainers) { ContainerId containerId = completedContainer.getContainerId(); + XTraceContext.clearThreadContext(); + containerId.joinContext(); LOG.debug("Container FINISHED: " + containerId); + XTraceContext.logEvent(CapacityScheduler.class, "Container Finished", "Container Finished", "Container ID", containerId); containerCompleted(getRMContainer(containerId), completedContainer, RMContainerEventType.FINISHED); + XTraceContext.clearThreadContext(); } if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource, @@ -651,6 +669,7 @@ private synchronized void nodeUpdate(RMNode rmNode) { @Override public void handle(SchedulerEvent event) { + event.joinContext(); switch(event.getType()) { case NODE_ADDED: { @@ -708,6 +727,9 @@ public void handle(SchedulerEvent event) { } private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { + XTraceContext.clearThreadContext(); + containerId.joinContext(); + // Get the application for the finished container ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); FiCaSchedulerApp application = getApplication(applicationAttemptId); @@ -723,6 +745,7 @@ private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode } application.containerLaunchedOnNode(containerId, node.getNodeID()); + XTraceContext.clearThreadContext(); } @Lock(FifoScheduler.class) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java index 2c9d67845a4f..d1c541a32db0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java @@ -313,4 +313,4 @@ synchronized public GetContainerStatusesResponse getContainerStatuses( nodeStatus.setNodeHealthStatus(nodeHealthStatus); return nodeStatus; } -} +} \ No newline at end of file From 77cf144bed85a4388ce52b3760a645e54ba4626a Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 2 Oct 2013 15:44:20 -0400 Subject: [PATCH 002/112] Removed unnecessary import --- .../src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java index 09383cd387c2..d4b6f3b3686c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java @@ -61,7 +61,6 @@ import edu.berkeley.xtrace.XTraceContext; import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceProcess; /** * RPC Engine for for protobuf based RPCs. From 9f994bf59bfbd5ff142651049b8f11b110834497 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 2 Oct 2013 15:46:39 -0400 Subject: [PATCH 003/112] Instrumented the WritableRpcEngine. I'm not sure where this is used (as opposed to the Protobuf engine) but Todd's htrace instrumentation chose to instrument this class too, so I might as well include it. --- .../apache/hadoop/ipc/WritableRpcEngine.java | 33 ++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java index 817b743b5689..441297dc60d9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java @@ -41,6 +41,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.*; +import java.util.Collection; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; /** An RpcEngine implementation for Writable data. */ @InterfaceStability.Evolving @@ -228,13 +231,27 @@ public Object invoke(Object proxy, Method method, Object[] args) startTime = Time.now(); } + XTraceContext.logEvent(RPC.class, "WritableRpcEngine", "RPC Client invoking remote method "+method.getName(), "ConnectionID", this.remoteId); + Collection start_context = XTraceContext.getThreadContext(); + try { // xtrace try + ObjectWritable value = (ObjectWritable) client.call(RPC.RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId); if (LOG.isDebugEnabled()) { long callTime = Time.now() - startTime; LOG.debug("Call: " + method.getName() + " " + callTime); } + + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(RPC.class, "WritableRpcEngine", "Client invocation of "+method.getName()+" successful"); + return value.get(); + + } catch (Exception e) {// xtrace catch + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(RPC.class, "WritableRpcEngine", "Remote invocation of "+method.getName()+" failed due to exception: "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } } /* close the IPC client that's responsible for this invoker's RPCs */ @@ -420,6 +437,10 @@ public Writable call(org.apache.hadoop.ipc.RPC.Server server, Invocation call = (Invocation)rpcRequest; if (server.verbose) log("Call: " + call); + + XTraceContext.logEvent(WritableRpcInvoker.class, "WritableRpcEngine", "Invoking method "+call.getMethodName()); + Collection start_context = XTraceContext.getThreadContext(); + try { // xtrace try // Verify writable rpc version if (call.getRpcVersion() != writableRpcVersion) { @@ -493,6 +514,10 @@ public Writable call(org.apache.hadoop.ipc.RPC.Server server, processingTime); if (server.verbose) log("Return: "+value); + + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(WritableRpcInvoker.class, "WritableRpcEngine", "Invocation of "+call.getMethodName()+" completed, responding to client"); + return new ObjectWritable(method.getReturnType(), value); } catch (InvocationTargetException e) { @@ -512,7 +537,13 @@ public Writable call(org.apache.hadoop.ipc.RPC.Server server, ioe.setStackTrace(e.getStackTrace()); throw ioe; } - } + + } catch (IOException e) { // xtrace catch + XTraceContext.joinContext(start_context); + XTraceContext.logEvent(WritableRpcInvoker.class, "ProtoBufRpcInvoker", "Failed to invoke method "+call.getMethodName()+": "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } + } } } From c2469a15e9dc3f691df660218f616e3b04624cb0 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 2 Oct 2013 15:47:28 -0400 Subject: [PATCH 004/112] Add XTrace metadata to the data transfer header protos --- .../hdfs/protocol/datatransfer/Receiver.java | 22 ++++++++++++++++++- .../src/main/proto/datatransfer.proto | 1 + 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index 910938ab44e0..200bdeaf328a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; @@ -33,6 +34,9 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; +import com.google.protobuf.ByteString; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; /** Receiver */ @InterfaceAudience.Private @@ -84,10 +88,21 @@ protected final void processOp(Op op) throws IOException { throw new IOException("Unknown op " + op + " in data stream"); } } + + private static void XTraceJoin(BaseHeaderProto header) { + if (header!=null && header.hasXtrace()) { + ByteString xbs = header.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); + if (xmd!=null && xmd.isValid()) + XTraceContext.setThreadContext(xmd); + } + XTraceContext.logEvent("HDFS Receiver", "Hello mateys"); + } /** Receive OP_READ_BLOCK */ private void opReadBlock() throws IOException { OpReadBlockProto proto = OpReadBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoin(proto.getHeader().getBaseHeader()); readBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -99,6 +114,7 @@ private void opReadBlock() throws IOException { /** Receive OP_WRITE_BLOCK */ private void opWriteBlock(DataInputStream in) throws IOException { final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoin(proto.getHeader().getBaseHeader()); writeBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -115,6 +131,7 @@ private void opWriteBlock(DataInputStream in) throws IOException { private void opTransferBlock(DataInputStream in) throws IOException { final OpTransferBlockProto proto = OpTransferBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoin(proto.getHeader().getBaseHeader()); transferBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -125,6 +142,7 @@ private void opTransferBlock(DataInputStream in) throws IOException { private void opRequestShortCircuitFds(DataInputStream in) throws IOException { final OpRequestShortCircuitAccessProto proto = OpRequestShortCircuitAccessProto.parseFrom(vintPrefixed(in)); + XTraceJoin(proto.getHeader()); requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken()), proto.getMaxVersion()); @@ -133,6 +151,7 @@ private void opRequestShortCircuitFds(DataInputStream in) throws IOException { /** Receive OP_REPLACE_BLOCK */ private void opReplaceBlock(DataInputStream in) throws IOException { OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoin(proto.getHeader()); replaceBlock(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken()), proto.getDelHint(), @@ -142,6 +161,7 @@ private void opReplaceBlock(DataInputStream in) throws IOException { /** Receive OP_COPY_BLOCK */ private void opCopyBlock(DataInputStream in) throws IOException { OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoin(proto.getHeader()); copyBlock(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken())); } @@ -149,7 +169,7 @@ private void opCopyBlock(DataInputStream in) throws IOException { /** Receive OP_BLOCK_CHECKSUM */ private void opBlockChecksum(DataInputStream in) throws IOException { OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in)); - + XTraceJoin(proto.getHeader()); blockChecksum(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken())); } 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..93716ea80421 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 { From 629774f864edcbf84b196e799587f9b0291ceea9 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 2 Oct 2013 15:47:43 -0400 Subject: [PATCH 005/112] Start tracing commands when initiated from the filesystem. --- .../src/main/java/org/apache/hadoop/fs/FsShell.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java index 0db1f9e43177..cf4c861b2571 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java @@ -34,6 +34,10 @@ import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import com.google.common.base.Joiner; + +import edu.berkeley.xtrace.XTraceContext; + /** Provide command line access to a FileSystem. */ @InterfaceAudience.Private public class FsShell extends Configured implements Tool { @@ -247,22 +251,26 @@ public int run(String argv[]) throws Exception { } else { String cmd = argv[0]; Command instance = null; + XTraceContext.startTrace("FsShell", "Executing Command", Joiner.on(" ").join(argv)); try { instance = commandFactory.getInstance(cmd); if (instance == null) { throw new UnknownCommandException(); } exitCode = instance.run(Arrays.copyOfRange(argv, 1, argv.length)); + XTraceContext.logEvent("FsShell", "Finished executing command"); } catch (IllegalArgumentException e) { displayError(cmd, e.getLocalizedMessage()); if (instance != null) { printInstanceUsage(System.err, instance); } + XTraceContext.logEvent("FsShell", "Command failed due to illegal argument"); } catch (Exception e) { // instance.run catches IOE, so something is REALLY wrong if here LOG.debug("Error", e); displayError(cmd, "Fatal internal error"); e.printStackTrace(System.err); + XTraceContext.logEvent("FsShell", "Fatal internal error", "Message", e.getMessage()); } } return exitCode; From 1a39a7fdd798d925af4ff40a449f2f1506c47f50 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 2 Oct 2013 17:16:01 -0400 Subject: [PATCH 006/112] Changed the DataNode receiver to start traces or join traces. --- .../hdfs/protocol/datatransfer/Receiver.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index 200bdeaf328a..03776e1ab16b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -89,20 +89,20 @@ protected final void processOp(Op op) throws IOException { } } - private static void XTraceJoin(BaseHeaderProto header) { + private static void XTraceJoinStart(String type, BaseHeaderProto header) { if (header!=null && header.hasXtrace()) { ByteString xbs = header.getXtrace(); XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); if (xmd!=null && xmd.isValid()) XTraceContext.setThreadContext(xmd); } - XTraceContext.logEvent("HDFS Receiver", "Hello mateys"); + XTraceContext.startTrace("DataNode", type); } /** Receive OP_READ_BLOCK */ private void opReadBlock() throws IOException { OpReadBlockProto proto = OpReadBlockProto.parseFrom(vintPrefixed(in)); - XTraceJoin(proto.getHeader().getBaseHeader()); + XTraceJoinStart("ReadBlock", proto.getHeader().getBaseHeader()); readBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -114,7 +114,7 @@ private void opReadBlock() throws IOException { /** Receive OP_WRITE_BLOCK */ private void opWriteBlock(DataInputStream in) throws IOException { final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in)); - XTraceJoin(proto.getHeader().getBaseHeader()); + XTraceJoinStart("WriteBlock", proto.getHeader().getBaseHeader()); writeBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -131,7 +131,7 @@ private void opWriteBlock(DataInputStream in) throws IOException { private void opTransferBlock(DataInputStream in) throws IOException { final OpTransferBlockProto proto = OpTransferBlockProto.parseFrom(vintPrefixed(in)); - XTraceJoin(proto.getHeader().getBaseHeader()); + XTraceJoinStart("TransferBlock", proto.getHeader().getBaseHeader()); transferBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -142,7 +142,7 @@ private void opTransferBlock(DataInputStream in) throws IOException { private void opRequestShortCircuitFds(DataInputStream in) throws IOException { final OpRequestShortCircuitAccessProto proto = OpRequestShortCircuitAccessProto.parseFrom(vintPrefixed(in)); - XTraceJoin(proto.getHeader()); + XTraceJoinStart("RequestShortCircuitFds", proto.getHeader()); requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken()), proto.getMaxVersion()); @@ -151,7 +151,7 @@ private void opRequestShortCircuitFds(DataInputStream in) throws IOException { /** Receive OP_REPLACE_BLOCK */ private void opReplaceBlock(DataInputStream in) throws IOException { OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in)); - XTraceJoin(proto.getHeader()); + XTraceJoinStart("ReplaceBlock", proto.getHeader()); replaceBlock(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken()), proto.getDelHint(), @@ -161,7 +161,7 @@ private void opReplaceBlock(DataInputStream in) throws IOException { /** Receive OP_COPY_BLOCK */ private void opCopyBlock(DataInputStream in) throws IOException { OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in)); - XTraceJoin(proto.getHeader()); + XTraceJoinStart("CopyBlock", proto.getHeader()); copyBlock(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken())); } @@ -169,7 +169,7 @@ private void opCopyBlock(DataInputStream in) throws IOException { /** Receive OP_BLOCK_CHECKSUM */ private void opBlockChecksum(DataInputStream in) throws IOException { OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in)); - XTraceJoin(proto.getHeader()); + XTraceJoinStart("BlockChecksum", proto.getHeader()); blockChecksum(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken())); } From 7fb8922ffba3148b1c0ddf124a7e743db1732099 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 3 Oct 2013 15:11:32 -0400 Subject: [PATCH 007/112] Added XTrace metadata to the block and packet transfer headers --- .../datatransfer/DataTransferProtoUtil.java | 11 ++++++++--- .../hdfs/protocol/datatransfer/PacketHeader.java | 16 ++++++++++++++++ .../src/main/proto/datatransfer.proto | 1 + 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java index 6be3810c9187..fa5537d16a5a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java @@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; @@ -30,6 +31,8 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; +import com.google.protobuf.ByteString; +import edu.berkeley.xtrace.XTraceContext; /** @@ -78,9 +81,11 @@ static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk, static BaseHeaderProto buildBaseHeader(ExtendedBlock blk, Token blockToken) { - return BaseHeaderProto.newBuilder() + Builder header = BaseHeaderProto.newBuilder() .setBlock(PBHelper.convert(blk)) - .setToken(PBHelper.convert(blockToken)) - .build(); + .setToken(PBHelper.convert(blockToken)); + if (XTraceContext.isValid()) + header.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + return header.build(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java index c9966a71a50d..67216e06924a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java @@ -31,6 +31,11 @@ import com.google.common.primitives.Shorts; import com.google.common.primitives.Ints; import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.ByteString; + +import edu.berkeley.xtrace.TaskID; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; /** * Header data for each packet that goes through the read/write pipelines. @@ -58,6 +63,7 @@ public class PacketHeader { .setLastPacketInBlock(false) .setDataLen(0) .setSyncBlock(false) + .setXtrace(ByteString.copyFrom(new XTraceMetadata(new TaskID(8), 0L).pack())) .build().getSerializedSize(); public static final int PKT_LENGTHS_LEN = Ints.BYTES + Shorts.BYTES; @@ -82,6 +88,9 @@ public PacketHeader(int packetLen, long offsetInBlock, long seqno, .setSeqno(seqno) .setLastPacketInBlock(lastPacketInBlock) .setDataLen(dataLen); + + if (XTraceContext.isValid()) + builder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); if (syncBlock) { // Only set syncBlock if it is specified. @@ -117,6 +126,13 @@ public int getPacketLen() { public boolean getSyncBlock() { return proto.getSyncBlock(); } + + public void joinXTraceContext() { + ByteString xbs = proto.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); + if (xmd.isValid()) + XTraceContext.joinContext(xmd); + } @Override public String toString() { 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 93716ea80421..b4c829bbfa01 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto @@ -140,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 { From a9c3ad7c492bad8b8caa5a6f909cbea0160dde19 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 3 Oct 2013 15:11:52 -0400 Subject: [PATCH 008/112] Instrumented the sender/receiver for downloading files from HDFS --- .../apache/hadoop/hdfs/RemoteBlockReader.java | 4 ++++ .../hadoop/hdfs/RemoteBlockReader2.java | 10 +++++++++ .../protocol/datatransfer/PacketReceiver.java | 18 ++++++++++++++++ .../hdfs/server/datanode/BlockSender.java | 21 ++++++++++++++++++- 4 files changed, 52 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index 61ce3eab157a..7754a12b1cf0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -43,6 +43,8 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; +import edu.berkeley.xtrace.XTraceContext; + /** * @deprecated this is an old implementation that is being left around @@ -383,6 +385,8 @@ public static RemoteBlockReader newBlockReader(String file, DatanodeID datanodeID, PeerCache peerCache) throws IOException { + XTraceContext.logEvent(RemoteBlockReader2.class, "RemoteBlockReader2", "Reading remote block using unsupported block reader", "file", file, "BlockName", block.getBlockName()); + // in and out will be closed when sock is closed (by the caller) final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(peer.getOutputStream())); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java index 1ba6b55da8f3..8839ad6adf7f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java @@ -50,6 +50,8 @@ import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; + /** * This is a wrapper around connection to datanode * and understands checksum, offset etc. @@ -376,6 +378,9 @@ public static BlockReader newBlockReader(String file, String clientName, Peer peer, DatanodeID datanodeID, PeerCache peerCache) throws IOException { + XTraceContext.logEvent(RemoteBlockReader2.class, "RemoteBlockReader2", "Reading remote block", "file", file, "BlockName", block.getBlockName()); + try { // xtrace try + // in and out will be closed when sock is closed (by the caller) final DataOutputStream out = new DataOutputStream(new BufferedOutputStream( peer.getOutputStream())); @@ -409,6 +414,11 @@ public static BlockReader newBlockReader(String file, return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(), checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID, peerCache); + + } catch (IOException e) { + XTraceContext.logEvent(RemoteBlockReader2.class, "RemoteBlockReader2", "IOException reading remote block", "Message", e.getMessage()); + throw e; + } } static void checkSuccess( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java index cc2d17974a4e..66a6eb625c7c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java @@ -23,6 +23,7 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; +import java.util.Collection; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -33,6 +34,9 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** * Class to handle reading packets one-at-a-time from the wire. * These packets are used both for reading and writing data to/from @@ -51,6 +55,8 @@ public class PacketReceiver implements Closeable { private static final DirectBufferPool bufferPool = new DirectBufferPool(); private final boolean useDirectBuffers; + + private Collection previous_read_context = XTraceContext.getThreadContext(); /** * The entirety of the most recently read packet. @@ -127,6 +133,10 @@ private void doRead(ReadableByteChannel ch, InputStream in) // CHECKSUMS: the crcs for the data chunk. May be missing if // checksums were not requested // DATA the actual block data + XTraceContext.joinContext(previous_read_context); + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Reading packet"); + try { // xtrace try + Preconditions.checkState(curHeader == null || !curHeader.isLastPacketInBlock()); curPacketBuf.clear(); @@ -179,6 +189,9 @@ private void doRead(ReadableByteChannel ch, InputStream in) curHeader = new PacketHeader(); } curHeader.setFieldsFromData(dataPlusChecksumLen, headerBuf); + curHeader.joinXTraceContext(); + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Finished reading packet"); + previous_read_context = XTraceContext.getThreadContext(); // Compute the sub-slices of the packet int checksumLen = dataPlusChecksumLen - curHeader.getDataLen(); @@ -189,6 +202,11 @@ private void doRead(ReadableByteChannel ch, InputStream in) } reslicePacket(headerLen, checksumLen, curHeader.getDataLen()); + + } catch (IOException e) { // xtrace catch + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Exception reading packet", "Message", e.getMessage()); + throw e; + } } /** 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..f3749e617307 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,8 @@ import com.google.common.base.Preconditions; +import edu.berkeley.xtrace.XTraceContext; + /** * Reads a block from the disk and sends it to a recipient. * @@ -454,6 +456,8 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out, int packetLen = dataLen + checksumDataLen + 4; boolean lastDataPacket = offset + dataLen == endOffset && dataLen > 0; + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Sending packet", "dataLen", dataLen, "numChunks", numChunks, "packetLen", packetLen); + // The packet buffer is organized as follows: // _______HHHHCCCCD?D?D?D? // ^ ^ @@ -515,6 +519,7 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out, // normal transfer out.write(buf, headerOff, dataOff + dataLen - headerOff); } + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Packet send complete"); } catch (IOException e) { if (e instanceof SocketTimeoutException) { /* @@ -523,6 +528,7 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out, * the socket open). */ LOG.info("exception: ", e); + XTraceContext.logEvent(BlockSender.class, "BlockSender", "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 +543,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); } + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Exception", "Message", ioem); } throw ioeToSocketException(e); } @@ -625,10 +632,14 @@ 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" ); } + + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Sending Block", "BlockName", block.getBlockName()); + try { // xtrace try + initialOffset = offset; long totalRead = 0; OutputStream streamForSendChunks = out; @@ -688,6 +699,9 @@ long sendBlock(DataOutputStream out, OutputStream baseStream, } sentEntireByteRange = true; + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Entire block sent", "totalRead", totalRead, "initialOffset", initialOffset); + } else { + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Block send interrupted", "totalRead", totalRead, "initialOffset", initialOffset); } } finally { if (clientTraceFmt != null) { @@ -698,6 +712,11 @@ long sendBlock(DataOutputStream out, OutputStream baseStream, close(); } return totalRead; + + } catch (IOException e) { // xtrace catch + XTraceContext.logEvent(BlockSender.class, "BlockSender", "IOException sending block", "Message", e.getMessage()); + throw e; + } } /** From 494f1e8c1a62ca13d418644d4ba4e7eb3d801e75 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 4 Oct 2013 18:50:08 -0400 Subject: [PATCH 009/112] Add check for xtrace metadata --- .../hadoop/hdfs/protocol/datatransfer/PacketHeader.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java index 67216e06924a..9437862620f0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java @@ -127,6 +127,10 @@ public boolean getSyncBlock() { return proto.getSyncBlock(); } + public boolean hasXTraceContext() { + return proto.hasXtrace(); + } + public void joinXTraceContext() { ByteString xbs = proto.getXtrace(); XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); From 17848a460541eaeede65b47505fb3b58385203c1 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 4 Oct 2013 18:51:33 -0400 Subject: [PATCH 010/112] Move start trace to the DataXceiver and add a little instrumentation for writeblock. --- .../hdfs/protocol/datatransfer/Receiver.java | 1 - .../hadoop/hdfs/server/datanode/DataXceiver.java | 15 +++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index 03776e1ab16b..a950b177e24e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -96,7 +96,6 @@ private static void XTraceJoinStart(String type, BaseHeaderProto header) { if (xmd!=null && xmd.isValid()) XTraceContext.setThreadContext(xmd); } - XTraceContext.startTrace("DataNode", type); } /** Receive OP_READ_BLOCK */ 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..40255e884a80 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 @@ -76,6 +76,8 @@ import com.google.protobuf.ByteString; +import edu.berkeley.xtrace.XTraceContext; + /** * Thread for processing incoming/outgoing data stream. @@ -220,12 +222,14 @@ public void run() { opStartTime = now(); processOp(op); ++opsProcessed; + XTraceContext.clearThreadContext(); } 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); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Error processing", "Message", t.getMessage()); } finally { if (LOG.isDebugEnabled()) { LOG.debug(datanode.getDisplayName() + ":Number of active connections is: " @@ -241,6 +245,8 @@ public void run() { public void requestShortCircuitFds(final ExtendedBlock blk, final Token token, int maxVersion) throws IOException { + XTraceContext.startTrace("DataNode", "RequestShortCircuitFds"); + updateCurrentThreadName("Passing file descriptors for block " + blk); BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder(); FileInputStream fis[] = null; @@ -300,6 +306,7 @@ public void readBlock(final ExtendedBlock block, final long blockOffset, final long length, final boolean sendChecksum) throws IOException { + XTraceContext.startTrace("DataNode", "ReadBlock"); previousOpClientName = clientName; OutputStream baseStream = getOutputStream(); @@ -394,6 +401,7 @@ public void writeBlock(final ExtendedBlock block, final long maxBytesRcvd, final long latestGenerationStamp, DataChecksum requestedChecksum) throws IOException { + XTraceContext.startTrace("DataNode", "WriteBlock"); previousOpClientName = clientname; updateCurrentThreadName("Receiving block " + block); final boolean isDatanode = clientname.length() == 0; @@ -467,6 +475,7 @@ public void writeBlock(final ExtendedBlock block, if (LOG.isDebugEnabled()) { LOG.debug("Connecting to datanode " + mirrorNode); } + XTraceContext.logEvent(DataXceiver.class, "DataXceiver writeBlock", "Connecting to mirror node", "mirrorNode", mirrorNode); mirrorTarget = NetUtils.createSocketAddr(mirrorNode); mirrorSock = datanode.newSocket(); try { @@ -513,6 +522,7 @@ public void writeBlock(final ExtendedBlock block, " from downstream datanode with firstbadlink as " + firstBadLink); } + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Got response for connect ack from downstream datanode"); } } catch (IOException e) { @@ -545,6 +555,7 @@ public void writeBlock(final ExtendedBlock block, // send connect-ack to source for clients and not transfer-RBW/Finalized if (isClient && !isTransfer) { + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Forwarding connect ack to upstream"); if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) { LOG.info("Datanode " + targets.length + " forwarding connect ack to upstream firstbadlink is " + @@ -613,6 +624,7 @@ public void transferBlock(final ExtendedBlock blk, final Token blockToken, final String clientName, final DatanodeInfo[] targets) throws IOException { + XTraceContext.startTrace("DataNode", "TransferBlock"); checkAccess(socketOut, true, blk, blockToken, Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY); previousOpClientName = clientName; @@ -631,6 +643,7 @@ public void transferBlock(final ExtendedBlock blk, @Override public void blockChecksum(final ExtendedBlock block, final Token blockToken) throws IOException { + XTraceContext.startTrace("DataNode", "BlockChecksum"); final DataOutputStream out = new DataOutputStream( getOutputStream()); checkAccess(out, true, block, blockToken, @@ -683,6 +696,7 @@ public void blockChecksum(final ExtendedBlock block, @Override public void copyBlock(final ExtendedBlock block, final Token blockToken) throws IOException { + XTraceContext.startTrace("DataNode", "CopyBlock"); updateCurrentThreadName("Copying block " + block); // Read in the header if (datanode.isBlockTokenEnabled) { @@ -758,6 +772,7 @@ public void replaceBlock(final ExtendedBlock block, final Token blockToken, final String delHint, final DatanodeInfo proxySource) throws IOException { + XTraceContext.startTrace("DataNode", "ReplaceBlock"); updateCurrentThreadName("Replacing block " + block + " from " + delHint); /* read header */ From 6c04b8126f517da779bd319df4185904e2a78847 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 4 Oct 2013 18:52:18 -0400 Subject: [PATCH 011/112] Add XTrace instrumentation to pipeline write acks --- .../protocol/datatransfer/PipelineAck.java | 28 ++++++++++-- .../hdfs/server/datanode/BlockReceiver.java | 44 ++++++++++++++++++- .../src/main/proto/datatransfer.proto | 1 + 3 files changed, 69 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java index b743e29f217b..14deeef2d008 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java @@ -28,10 +28,16 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.Builder; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; +import com.google.protobuf.ByteString; import com.google.protobuf.TextFormat; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceMetadataCollection; + /** Pipeline Acknowledgment **/ @InterfaceAudience.Private @InterfaceStability.Evolving @@ -59,11 +65,13 @@ public PipelineAck(long seqno, Status[] replies) { * @param downstreamAckTimeNanos ack RTT in nanoseconds, 0 if no next DN in pipeline */ public PipelineAck(long seqno, Status[] replies, long downstreamAckTimeNanos) { - proto = PipelineAckProto.newBuilder() + Builder builder = PipelineAckProto.newBuilder() .setSeqno(seqno) .addAllStatus(Arrays.asList(replies)) - .setDownstreamAckTimeNanos(downstreamAckTimeNanos) - .build(); + .setDownstreamAckTimeNanos(downstreamAckTimeNanos); + if (XTraceContext.isValid()) + builder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + proto = builder.build(); } /** @@ -97,6 +105,20 @@ public Status getReply(int i) { public long getDownstreamAckTimeNanos() { return proto.getDownstreamAckTimeNanos(); } + + public boolean hasXtrace() { + return proto.hasXtrace(); + } + + public void joinXtraceContext() { + if (proto.hasXtrace()) { + ByteString xbs = proto.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), + 0, xbs.size()); + if (xmd.isValid()) + XTraceContext.joinContext(xmd); + } + } /** * Check if this ack contains error status 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..30f199801ed3 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 @@ -29,6 +29,7 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collection; import java.util.LinkedList; import java.util.zip.Checksum; @@ -53,6 +54,9 @@ import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.StringUtils; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + /** 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. @@ -263,6 +267,7 @@ public void close() throws IOException { boolean measuredFlushTime = false; // close checksum file try { + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Closing checksum file"); if (checksumOut != null) { long flushStartNanos = System.nanoTime(); checksumOut.flush(); @@ -277,6 +282,7 @@ public void close() throws IOException { checksumOut.close(); checksumOut = null; } + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Checksum file closed"); } catch(IOException e) { ioe = e; } @@ -285,6 +291,7 @@ public void close() throws IOException { } // close block file try { + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Closing block file"); if (out != null) { long flushStartNanos = System.nanoTime(); out.flush(); @@ -299,6 +306,7 @@ public void close() throws IOException { out.close(); out = null; } + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Block file closed"); } catch (IOException e) { ioe = e; } @@ -336,6 +344,7 @@ void flushOrSync(boolean isSync) throws IOException { long flushStartNanos = System.nanoTime(); out.flush(); long flushEndNanos = System.nanoTime(); + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushed file output stream"); if (isSync && (out instanceof FileOutputStream)) { long fsyncStartNanos = flushEndNanos; ((FileOutputStream)out).getChannel().force(true); @@ -425,6 +434,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 +568,7 @@ private int receivePacket() throws IOException { int numBytesToDisk = (int)(offsetInBlock-onDiskLen); // Write data to disk. + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Writing data 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 @@ -648,6 +659,9 @@ void receiveBlock( DataOutputStream replyOut, // output to previous datanode String mirrAddr, DataTransferThrottler throttlerArg, DatanodeInfo[] downstreams) throws IOException { + + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Receiving Block"); + try { // xtrace try syncOnClose = datanode.getDnConf().syncOnClose; boolean responderClosed = false; @@ -721,6 +735,10 @@ void receiveBlock( responder = null; } } + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Finished receiving block"); + } catch (IOException e) { // xtrace catch + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "IOException receiving block", "Message", e.getMessage()); + } } /** Cleanup a partial block @@ -834,7 +852,10 @@ 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; + + /** XTrace context when packet responder was created */ + private Collection xtrace = XTraceContext.getThreadContext(); @Override public String toString() { @@ -898,6 +919,7 @@ public synchronized void close() { } running = false; notifyAll(); + XTraceContext.joinContext(xtrace); // rejoin the end xtrace context } /** @@ -906,6 +928,7 @@ public synchronized void close() { */ @Override public void run() { + XTraceContext.setThreadContext(xtrace); // set the start xtrace context boolean lastPacketInBlock = false; final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0; while (running && datanode.shouldRun && !lastPacketInBlock) { @@ -923,11 +946,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(); + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Received ack", "seqno", seqno); } if (seqno != PipelineAck.UNKOWN_SEQNO || type == PacketResponderType.LAST_IN_PIPELINE) { @@ -943,7 +969,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 +1007,7 @@ public void run() { // and wait for the client to shut down the pipeline mirrorError = true; LOG.info(myString, ioe); + XTraceContext.logEvent(PacketResponder.class, "PacketResponder", "IOException from mirror", "Message", ioe.getMessage()); } } @@ -991,6 +1020,7 @@ public void run() { * will detect that this datanode is bad, and rightly so. */ LOG.info(myString + ": Thread is interrupted."); + XTraceContext.logEvent(PacketResponder.class, "PacketResponder", "Thread is interrupted"); running = false; continue; } @@ -1043,6 +1073,7 @@ public void run() { continue; } } + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Acknowledging packet", "seqno", seqno, "AckTimeNanos", totalAckTimeNanos); PipelineAck replyAck = new PipelineAck(expected, replies, totalAckTimeNanos); if (replyAck.isSuccess() && @@ -1060,6 +1091,7 @@ public void run() { removeAckHead(); // update bytes acked } + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Packet ack sent."); // terminate after sending response if this node detected // a checksum error if (myStatus == Status.ERROR_CHECKSUM) { @@ -1070,6 +1102,7 @@ public void run() { } } catch (IOException e) { LOG.warn("IOException in BlockReceiver.run(): ", e); + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "IOException in BlockReceiver.run", "Message", e.getMessage()); if (running) { try { datanode.checkDiskError(e); // may throw an exception here @@ -1088,6 +1121,8 @@ public void run() { running = false; receiverThread.interrupt(); } + } finally { + xtrace = XTraceContext.getThreadContext(); } } LOG.info(myString + " terminating"); @@ -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 Collection xtrace = XTraceContext.getThreadContext(); 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() { + XTraceContext.joinContext(xtrace); + } @Override public String toString() { 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 b4c829bbfa01..83685d186041 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto @@ -158,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 } /** From 58e6a9b9da8f005b68619806674fa5ab540bcad5 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 4 Oct 2013 18:52:52 -0400 Subject: [PATCH 012/112] Instrument code that mirrors data when pipelining to in-place modify the xtrace metadata --- .../protocol/datatransfer/PacketReceiver.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java index 66a6eb625c7c..935b44a4e83d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java @@ -215,9 +215,34 @@ private void doRead(ReadableByteChannel ch, InputStream in) public void mirrorPacketTo(DataOutputStream mirrorOut) throws IOException { Preconditions.checkState(!useDirectBuffers, "Currently only supported for non-direct buffers"); + + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Mirroring packet"); + try { // xtrace try + + updateHeaderXTrace(); mirrorOut.write(curPacketBuf.array(), curPacketBuf.arrayOffset(), curPacketBuf.remaining()); + + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Packet mirrored successfully"); + } catch (IOException e) { // xtrace catch + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Exception writing block to mirror", "Message", e.getMessage()); + } + } + + /** + * This updates the XTrace metadata in the packet header to the current context + */ + private void updateHeaderXTrace() { + // Only update context if there was a previous one, and we assume they have the exact + // same length, so we can just drop in a new packet header. + if (XTraceContext.isValid() && curHeader.hasXTraceContext()) { + PacketHeader newHeader = new PacketHeader(curHeader.getPacketLen(), curHeader.getOffsetInBlock(), + curHeader.getSeqno(), curHeader.isLastPacketInBlock(), curHeader.getDataLen(), + curHeader.getSyncBlock()); + curPacketBuf.position(0); + newHeader.putInBuffer(curPacketBuf); + } } From 726d9a0e459220bca9d10c038c240965c373c5fa Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 4 Oct 2013 19:13:08 -0400 Subject: [PATCH 013/112] Removed unnecessary instrumentation --- .../hadoop/hdfs/protocol/datatransfer/PacketReceiver.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java index 935b44a4e83d..754b4d22739c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java @@ -56,8 +56,6 @@ public class PacketReceiver implements Closeable { private static final DirectBufferPool bufferPool = new DirectBufferPool(); private final boolean useDirectBuffers; - private Collection previous_read_context = XTraceContext.getThreadContext(); - /** * The entirety of the most recently read packet. * The first PKT_LENGTHS_LEN bytes of this buffer are the @@ -133,7 +131,6 @@ private void doRead(ReadableByteChannel ch, InputStream in) // CHECKSUMS: the crcs for the data chunk. May be missing if // checksums were not requested // DATA the actual block data - XTraceContext.joinContext(previous_read_context); XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Reading packet"); try { // xtrace try @@ -191,7 +188,6 @@ private void doRead(ReadableByteChannel ch, InputStream in) curHeader.setFieldsFromData(dataPlusChecksumLen, headerBuf); curHeader.joinXTraceContext(); XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Finished reading packet"); - previous_read_context = XTraceContext.getThreadContext(); // Compute the sub-slices of the packet int checksumLen = dataPlusChecksumLen - curHeader.getDataLen(); From cec72b71e9b71cb8735636cfeecb9da002c860b7 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 4 Oct 2013 19:14:16 -0400 Subject: [PATCH 014/112] Minor mistakes in the xtrace log statements --- .../apache/hadoop/hdfs/server/datanode/BlockReceiver.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 30f199801ed3..bed50a15e0a0 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 @@ -568,7 +568,7 @@ private int receivePacket() throws IOException { int numBytesToDisk = (int)(offsetInBlock-onDiskLen); // Write data to disk. - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Writing data to file output stream", "start", startByteToDisk, "size", numBytesToDisk); + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "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 @@ -1073,7 +1073,7 @@ public void run() { continue; } } - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Acknowledging packet", "seqno", seqno, "AckTimeNanos", totalAckTimeNanos); + XTraceContext.logEvent(PacketReceiver.class, "PacketResponder", "Acknowledging packet", "seqno", seqno, "AckTimeNanos", totalAckTimeNanos); PipelineAck replyAck = new PipelineAck(expected, replies, totalAckTimeNanos); if (replyAck.isSuccess() && @@ -1102,7 +1102,7 @@ public void run() { } } catch (IOException e) { LOG.warn("IOException in BlockReceiver.run(): ", e); - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "IOException in BlockReceiver.run", "Message", e.getMessage()); + XTraceContext.logEvent(PacketReceiver.class, "PacketResponder", "IOException in BlockReceiver.run", "Message", e.getMessage()); if (running) { try { datanode.checkDiskError(e); // may throw an exception here From 49cc05b74837575752b0b53adc047bb67db56ac0 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 4 Oct 2013 20:05:07 -0400 Subject: [PATCH 015/112] Tiny bits of additional instrumentation --- .../apache/hadoop/hdfs/server/datanode/BlockReceiver.java | 5 +++-- .../org/apache/hadoop/hdfs/server/datanode/DataXceiver.java | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) 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 bed50a15e0a0..a4fedb5b2360 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 @@ -267,7 +267,7 @@ public void close() throws IOException { boolean measuredFlushTime = false; // close checksum file try { - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Closing checksum file"); + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushing checksum file"); if (checksumOut != null) { long flushStartNanos = System.nanoTime(); checksumOut.flush(); @@ -291,7 +291,7 @@ public void close() throws IOException { } // close block file try { - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Closing block file"); + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushing block file"); if (out != null) { long flushStartNanos = System.nanoTime(); out.flush(); @@ -1123,6 +1123,7 @@ public void run() { } } finally { xtrace = XTraceContext.getThreadContext(); + XTraceContext.clearThreadContext(); } } LOG.info(myString + " terminating"); 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 40255e884a80..14b2e8b8e9f0 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 @@ -580,6 +580,7 @@ public void writeBlock(final ExtendedBlock block, if (LOG.isTraceEnabled()) { LOG.trace("TRANSFER: send close-ack"); } + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sending close ack"); writeResponse(SUCCESS, null, replyOut); } } From 35e2e17410a2c02ee6f1afc4607ef594069dbace Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 4 Oct 2013 20:13:21 -0400 Subject: [PATCH 016/112] Changed location of 'close' log messages, since close can be called multiple times, we only want it logged when the stream is first closed. --- .../apache/hadoop/hdfs/server/datanode/BlockReceiver.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 a4fedb5b2360..6b22e63005a2 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 @@ -267,8 +267,8 @@ public void close() throws IOException { boolean measuredFlushTime = false; // close checksum file try { - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushing checksum file"); if (checksumOut != null) { + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushing checksum file"); long flushStartNanos = System.nanoTime(); checksumOut.flush(); long flushEndNanos = System.nanoTime(); @@ -281,8 +281,8 @@ public void close() throws IOException { measuredFlushTime = true; checksumOut.close(); checksumOut = null; + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Checksum file closed"); } - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Checksum file closed"); } catch(IOException e) { ioe = e; } @@ -291,8 +291,8 @@ public void close() throws IOException { } // close block file try { - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushing block file"); if (out != null) { + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushing block file"); long flushStartNanos = System.nanoTime(); out.flush(); long flushEndNanos = System.nanoTime(); @@ -305,8 +305,8 @@ public void close() throws IOException { measuredFlushTime = true; out.close(); out = null; + XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Block file closed"); } - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Block file closed"); } catch (IOException e) { ioe = e; } From ce8a39ebab05238f4e166c11c7a9e646fe2afd8e Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 8 Oct 2013 18:24:29 -0400 Subject: [PATCH 017/112] Finished implementation of file writes from client side; mostly implementation in DFSOutputStream --- .../org/apache/hadoop/fs/FSOutputSummer.java | 9 ++ .../apache/hadoop/hdfs/DFSOutputStream.java | 89 +++++++++++++++++++ .../hdfs/server/datanode/DataXceiver.java | 4 + 3 files changed, 102 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java index 691fbfbd16d3..663649153c79 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java @@ -25,6 +25,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import edu.berkeley.xtrace.XTraceContext; + /** * This is a generic output stream for generating checksums for * data before it is written to the underlying stream @@ -95,6 +97,8 @@ public synchronized void write(int b) throws IOException { public synchronized void write(byte b[], int off, int len) throws IOException { + try { // xtrace try + checkClosed(); if (off < 0 || len < 0 || off > b.length - len) { @@ -103,6 +107,11 @@ public synchronized void write(byte b[], int off, int len) for (int n=0;n lastAckedXTraceContext = null; + private class Packet { long seqno; // sequencenumber of buffer in block long offsetInBlock; // offset in block @@ -178,6 +184,8 @@ private class Packet { int dataPos; private static final long HEART_BEAT_SEQNO = -1L; + + private Collection xtrace = null; /** * Create a heartbeat packet. @@ -215,6 +223,15 @@ private class Packet { dataStart = checksumStart + (chunksPerPkt * checksum.getChecksumSize()); dataPos = dataStart; maxChunks = chunksPerPkt; + rememberXTraceContext(); + } + + void joinXTraceContext() { + XTraceContext.joinContext(xtrace); + } + + void rememberXTraceContext() { + xtrace = XTraceContext.getThreadContext(); } void writeData(byte[] inarray, int off, int len) { @@ -350,6 +367,9 @@ public DatanodeInfo load(DatanodeInfo key) throws Exception { private boolean isHflushed = false; /** Append on an existing block? */ private final boolean isAppend; + + /** XTrace Context when streamer is created */ + private Collection xtrace = XTraceContext.getThreadContext(); /** * Default construction for file create @@ -435,6 +455,7 @@ private void endBlock() { if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Closing old block " + block); } + XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Closing block", "BlockName", block.getBlockName()); this.setName("DataStreamer for file " + src); closeResponder(); closeStream(); @@ -448,6 +469,9 @@ private void endBlock() { */ @Override public void run() { + XTraceContext.joinContext(xtrace); + try { //xtrace try + long lastPacket = Time.now(); while (!streamerClosed && dfsClient.clientRunning) { @@ -456,6 +480,7 @@ public void run() { try { response.close(); response.join(); + response.joinXTraceContext(); response = null; } catch (InterruptedException e) { } @@ -494,10 +519,13 @@ public void run() { } // get packet to be sent. if (dataQueue.isEmpty()) { + XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Sending heartbeat packet"); one = new Packet(); // heartbeat packet } else { one = dataQueue.getFirst(); // regular data packet } + one.joinXTraceContext(); + } assert one != null; @@ -506,12 +534,14 @@ public void run() { if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Allocating new block"); } + XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Allocating new block"); nodes = nextBlockOutputStream(src); initDataStreaming(); } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) { if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Append to block " + block); } + XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Appending to block", "BlockName", block.getBlockName()); setupPipelineForAppendOrRecovery(); initDataStreaming(); } @@ -543,11 +573,14 @@ public void run() { stage = BlockConstructionStage.PIPELINE_CLOSE; } + XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Sending packet"); + // send the packet synchronized (dataQueue) { // move packet from dataQueue to ackQueue if (!one.isHeartbeatPacket()) { dataQueue.removeFirst(); + one.rememberXTraceContext(); ackQueue.addLast(one); dataQueue.notifyAll(); } @@ -570,6 +603,9 @@ public void run() { } lastPacket = Time.now(); + XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Packet sent"); + one.rememberXTraceContext(); + if (one.isHeartbeatPacket()) { //heartbeat packet } @@ -606,6 +642,7 @@ public void run() { } } catch (Throwable e) { DFSClient.LOG.warn("DataStreamer Exception", e); + XTraceContext.logEvent(DataStreamer.class, "DataStreamer Exception", "Message", e.getMessage()); if (e instanceof IOException) { setLastException((IOException)e); } @@ -616,6 +653,10 @@ public void run() { } } closeInternal(); + + } finally { // xtrace finally + xtrace = XTraceContext.getThreadContext(); + } } private void closeInternal() { @@ -627,6 +668,10 @@ private void closeInternal() { dataQueue.notifyAll(); } } + + public void joinXTraceContext() { + XTraceContext.joinContext(xtrace); + } /* * close both streamer and DFSOutputStream, should be called only @@ -654,6 +699,7 @@ private void closeResponder() { response.join(); } catch (InterruptedException e) { } finally { + response.joinXTraceContext(); response = null; } } @@ -698,6 +744,8 @@ private class ResponseProcessor extends Daemon { private volatile boolean responderClosed = false; private DatanodeInfo[] targets = null; private boolean isLastPacketInBlock = false; + + private Collection xtrace = XTraceContext.getThreadContext(); ResponseProcessor (DatanodeInfo[] targets) { this.targets = targets; @@ -705,6 +753,8 @@ private class ResponseProcessor extends Daemon { @Override public void run() { + XTraceContext.joinContext(xtrace); + try { // xtrace try setName("ResponseProcessor for block " + block); PipelineAck ack = new PipelineAck(); @@ -719,6 +769,8 @@ public void run() { } long seqno = ack.getSeqno(); + ack.joinXtraceContext(); + XTraceContext.logEvent(ResponseProcessor.class, "ResponseProcessor", "Processing ACK", "seqno", seqno); // processes response status from datanodes. for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) { final Status reply = ack.getReply(i); @@ -742,6 +794,7 @@ public void run() { synchronized (dataQueue) { one = ackQueue.getFirst(); } + one.joinXTraceContext(); if (one.seqno != seqno) { throw new IOException("Responseprocessor: Expecting seqno " + " for block " + block + @@ -751,8 +804,10 @@ public void run() { // update bytesAcked block.setNumBytes(one.getLastByteOffsetBlock()); + XTraceContext.logEvent(ResponseProcessor.class, "ResponseProcessor", "Packet acknowledged"); synchronized (dataQueue) { lastAckedSeqno = seqno; + lastAckedXTraceContext = XTraceContext.getThreadContext(); ackQueue.removeFirst(); dataQueue.notifyAll(); } @@ -768,16 +823,25 @@ public void run() { } DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception " + " for block " + block, e); + XTraceContext.logEvent(ResponseProcessor.class, "ResponseProcessor", "Exception processing responses", "Message", e.getMessage()); responderClosed = true; } } } + + } finally { // xtrace finally + xtrace = XTraceContext.getThreadContext(); + } } void close() { responderClosed = true; this.interrupt(); } + + void joinXTraceContext() { + XTraceContext.joinContext(xtrace); + } } // If this stream has encountered any errors so far, shutdown @@ -899,6 +963,7 @@ private void addDatanode2ExistingPipeline() throws IOException { * - Append/Create: * + no transfer, let NameNode replicates the block. */ + XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Adding DataNode to Existing Pipeline"); if (!isAppend && lastAckedSeqno < 0 && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) { //no data have been written @@ -1102,6 +1167,7 @@ private DatanodeInfo[] nextBlockOutputStream(String client) throws IOException { // private boolean createBlockOutputStream(DatanodeInfo[] nodes, long newGS, boolean recoveryFlag) { + Status pipelineStatus = SUCCESS; String firstBadLink = ""; if (DFSClient.LOG.isDebugEnabled()) { @@ -1454,6 +1520,8 @@ private void queueCurrentPacket() { if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Queued packet " + currentPacket.seqno); } + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Packet added to output queue"); + currentPacket.rememberXTraceContext(); currentPacket = null; dataQueue.notifyAll(); } @@ -1464,6 +1532,7 @@ private void waitAndQueueCurrentPacket() throws IOException { // If queue is full, then wait till we have enough space while (!closed && dataQueue.size() + ackQueue.size() > MAX_PACKETS) { try { + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Output queues full, waiting for space..."); dataQueue.wait(); } catch (InterruptedException e) { // If we get interrupted while waiting to queue data, we still need to get rid @@ -1513,6 +1582,8 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] che ", chunksPerPacket=" + chunksPerPacket + ", bytesCurBlock=" + bytesCurBlock); } + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Allocating new packet", "seqno", currentPacket.seqno); + currentPacket.rememberXTraceContext(); } currentPacket.writeChecksum(checksum, 0, cklen); @@ -1520,6 +1591,10 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] che currentPacket.numChunks++; bytesCurBlock += len; + // Add in the current xtrace context + currentPacket.joinXTraceContext(); + currentPacket.rememberXTraceContext(); + // If packet is full, enqueue it for transmission // if (currentPacket.numChunks == currentPacket.maxChunks || @@ -1551,6 +1626,7 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] che // indicate the end of block and reset bytesCurBlock. // if (bytesCurBlock == blockSize) { + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Sending empty packet to indicate end of block"); currentPacket = new Packet(0, 0, bytesCurBlock); currentPacket.lastPacketInBlock = true; currentPacket.syncBlock = shouldSyncBlock; @@ -1618,6 +1694,12 @@ public void hsync(EnumSet syncFlags) throws IOException { */ private void flushOrSync(boolean isSync, EnumSet syncFlags) throws IOException { + if (isSync) + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "hysnc - Syncing all data to datanode disks"); + else + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "hflush - Flushing all data to datanode buffers"); + + dfsClient.checkOpen(); checkClosed(); try { @@ -1725,6 +1807,7 @@ private void flushOrSync(boolean isSync, EnumSet syncFlags) throw interrupt; } catch (IOException e) { DFSClient.LOG.warn("Error while syncing", e); + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Error while syncing", "Message", e.getMessage()); synchronized (this) { if (!closed) { lastException = new IOException("IOException flush:" + e); @@ -1785,10 +1868,12 @@ private void waitForAckedSeqno(long seqno) throws IOException { if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Waiting for ack for: " + seqno); } + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Waiting for ack", "seqno", seqno); synchronized (dataQueue) { while (!closed) { checkClosed(); if (lastAckedSeqno >= seqno) { + XTraceContext.joinContext(lastAckedXTraceContext); break; } try { @@ -1826,6 +1911,7 @@ private void closeThreads(boolean force) throws IOException { try { streamer.close(force); streamer.join(); + streamer.joinXTraceContext(); if (s != null) { s.close(); } @@ -1853,6 +1939,8 @@ public synchronized void close() throws IOException { } try { + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Closing stream"); + flushBuffer(); // flush from all upper layers if (currentPacket != null) { @@ -1874,6 +1962,7 @@ public synchronized void close() throws IOException { dfsClient.endFileLease(src); } finally { closed = true; + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Stream closed"); } } 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 14b2e8b8e9f0..bedc61ec308a 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 @@ -382,6 +382,8 @@ public void readBlock(final ExtendedBlock block, throw ioe; } finally { IOUtils.closeStream(blockSender); + + XTraceContext.logEvent(DataXceiver.class, "DataNode", "ReadBlock Complete"); } //update metrics @@ -613,6 +615,8 @@ public void writeBlock(final ExtendedBlock block, IOUtils.closeStream(replyOut); IOUtils.closeSocket(mirrorSock); IOUtils.closeStream(blockReceiver); + + XTraceContext.logEvent(DataXceiver.class, "DataNode", "WriteBlock Complete"); } //update metrics From 74d8f9140aeb4c2310ec73816fb13943979e4fbc Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 8 Oct 2013 18:55:20 -0400 Subject: [PATCH 018/112] Instrumentation bugfix --- .../hadoop/hdfs/server/datanode/BlockReceiver.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) 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 6b22e63005a2..453fdf4b4d9d 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 @@ -929,6 +929,8 @@ public synchronized void close() { @Override public void run() { XTraceContext.setThreadContext(xtrace); // set the start xtrace context + try { // xtrace try + boolean lastPacketInBlock = false; final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0; while (running && datanode.shouldRun && !lastPacketInBlock) { @@ -1091,7 +1093,7 @@ public void run() { removeAckHead(); // update bytes acked } - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Packet ack sent."); + XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Packet ack sent."); // terminate after sending response if this node detected // a checksum error if (myStatus == Status.ERROR_CHECKSUM) { @@ -1121,11 +1123,12 @@ public void run() { running = false; receiverThread.interrupt(); } - } finally { - xtrace = XTraceContext.getThreadContext(); - XTraceContext.clearThreadContext(); } } + } finally { // xtrace finally + xtrace = XTraceContext.getThreadContext(); + XTraceContext.clearThreadContext(); + } LOG.info(myString + " terminating"); } From d62da51702ea642d4cbda8671d44e1f8bd8f93c7 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 10 Oct 2013 14:42:32 -0400 Subject: [PATCH 019/112] Instrumented a couple more protobuf messages, added a utility class to put/retrieve XTrace contexts from those protobuf messages, and modified a bunch of classes to call the utility methods rather than the protobuf newBuilder methods. --- .../hadoop/hdfs/BlockReaderFactory.java | 2 + .../org/apache/hadoop/hdfs/DFSClient.java | 3 + .../apache/hadoop/hdfs/DFSOutputStream.java | 3 + .../apache/hadoop/hdfs/RemoteBlockReader.java | 4 +- .../hadoop/hdfs/RemoteBlockReader2.java | 5 +- .../hdfs/protocol/XTraceProtoUtils.java | 85 +++++++++++++++++++ .../hadoop/hdfs/server/balancer/Balancer.java | 2 + .../hdfs/server/datanode/DataXceiver.java | 16 +++- .../src/main/proto/datatransfer.proto | 3 + 9 files changed, 117 insertions(+), 6 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java index 197ab23b29fa..09655ec63a11 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; @@ -177,6 +178,7 @@ private static BlockReaderLocal newShortCircuitBlockReader( new DataInputStream(peer.getInputStream()); BlockOpResponseProto resp = BlockOpResponseProto.parseFrom( PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(resp); DomainSocket sock = peer.getDomainSocket(); switch (resp.getStatus()) { case SUCCESS: diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index c6aef2c929d0..19f867adf500 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -122,6 +122,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.Op; @@ -1789,6 +1790,7 @@ private static MD5MD5CRC32FileChecksum getFileChecksum(String src, final BlockOpResponseProto reply = BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(reply); if (reply.getStatus() != Status.SUCCESS) { if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) { @@ -1971,6 +1973,7 @@ private static Type inferChecksumTypeByReading( new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName, 0, 1, true); final BlockOpResponseProto reply = BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(reply); if (reply.getStatus() != Status.SUCCESS) { if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 72b48d641096..fcec7fdde93d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor; @@ -1021,6 +1022,7 @@ private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets, //ack BlockOpResponseProto response = BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(response); if (SUCCESS != response.getStatus()) { throw new IOException("Failed to add a datanode"); } @@ -1216,6 +1218,7 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, long newGS, PBHelper.vintPrefixed(blockReplyStream)); pipelineStatus = resp.getStatus(); firstBadLink = resp.getFirstBadLink(); + XTraceProtoUtils.join(resp); if (pipelineStatus != SUCCESS) { if (pipelineStatus == Status.ERROR_ACCESS_TOKEN) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index 7754a12b1cf0..655782d7367d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; @@ -385,7 +386,7 @@ public static RemoteBlockReader newBlockReader(String file, DatanodeID datanodeID, PeerCache peerCache) throws IOException { - XTraceContext.logEvent(RemoteBlockReader2.class, "RemoteBlockReader2", "Reading remote block using unsupported block reader", "file", file, "BlockName", block.getBlockName()); + XTraceContext.logEvent(RemoteBlockReader.class, "RemoteBlockReader", "Reading remote block using unsupported block reader", "file", file, "BlockName", block.getBlockName()); // in and out will be closed when sock is closed (by the caller) final DataOutputStream out = @@ -402,6 +403,7 @@ public static RemoteBlockReader newBlockReader(String file, BlockOpResponseProto status = BlockOpResponseProto.parseFrom( PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(status); RemoteBlockReader2.checkSuccess(status, peer, block, file); ReadOpChecksumInfoProto checksumInfo = status.getReadOpChecksumInfo(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java index 8839ad6adf7f..bea61ce3ece3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver; @@ -215,6 +216,7 @@ private void readNextPacket() throws IOException { // If we've now satisfied the whole client read, read one last packet // header, which should be empty if (bytesNeededToFinish <= 0) { + XTraceContext.logEvent(RemoteBlockReader2.class, "RemoteBlockReader2", "Block finished, reading trailing empty packet"); readTrailingEmptyPacket(); if (verifyChecksum) { sendReadResult(Status.CHECKSUM_OK); @@ -323,7 +325,7 @@ void sendReadResult(Status statusCode) { static void writeReadResult(OutputStream out, Status statusCode) throws IOException { - ClientReadStatusProto.newBuilder() + XTraceProtoUtils.newClientReadStatusProtoBuilder() .setStatus(statusCode) .build() .writeDelimitedTo(out); @@ -394,6 +396,7 @@ public static BlockReader newBlockReader(String file, BlockOpResponseProto status = BlockOpResponseProto.parseFrom( PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(status); checkSuccess(status, peer, block, file); ReadOpChecksumInfoProto checksumInfo = status.getReadOpChecksumInfo(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java new file mode 100644 index 000000000000..3887958e4937 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java @@ -0,0 +1,85 @@ +package org.apache.hadoop.hdfs.protocol; + +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto; + +import com.google.protobuf.ByteString; +import edu.berkeley.xtrace.XTraceContext; +import edu.berkeley.xtrace.XTraceMetadata; + +/** + * Contains some utility functions for XTrace instrumentation. Saves having to repeat + * instrumentation in loads of places in the code. + * @author jon + * + */ +public class XTraceProtoUtils { + + /** + * Shortcut method to create a new builder, then insert the current XTraceContext into it + * @return + */ + public static BlockOpResponseProto.Builder newBlockOpResponseProtoBuilder() { + BlockOpResponseProto.Builder b = BlockOpResponseProto.newBuilder(); + setXtrace(b); + return b; + } + + /** + * If the current XTraceContext is valid, sets it in the provided builder + * @param builder + */ + public static void setXtrace(BlockOpResponseProto.Builder builder) { + if (XTraceContext.isValid()) + builder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + } + + /** + * Joins an XTrace context if this message contains one + * @param p + */ + public static void join(BlockOpResponseProto p) { + if (!p.hasXtrace()) + return; + + ByteString xbs = p.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); + if (xmd.isValid()) + XTraceContext.joinContext(xmd); + } + + /** + * Shortcut method to create a new builder, then insert the current XTraceContext into it + * @return + */ + public static ClientReadStatusProto.Builder newClientReadStatusProtoBuilder() { + ClientReadStatusProto.Builder b = ClientReadStatusProto.newBuilder(); + setXtrace(b); + return b; + } + + /** + * If the current XTraceContext is valid, sets it in the provided builder + * @param builder + */ + public static void setXtrace(ClientReadStatusProto.Builder builder) { + if (XTraceContext.isValid()) + builder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + } + + /** + * Joins an XTrace context if this message contains one + * @param p + */ + public static void join(ClientReadStatusProto p) { + if (!p.hasXtrace()) + return; + + ByteString xbs = p.getXtrace(); + XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); + if (xmd.isValid()) + XTraceContext.joinContext(xmd); + } + + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java index cac8bf31bc78..d0b18665c645 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java @@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; @@ -392,6 +393,7 @@ private void sendRequest(DataOutputStream out) throws IOException { private void receiveResponse(DataInputStream in) throws IOException { BlockOpResponseProto response = BlockOpResponseProto.parseFrom( vintPrefixed(in)); + XTraceProtoUtils.join(response); if (response.getStatus() != Status.SUCCESS) { if (response.getStatus() == Status.ERROR_ACCESS_TOKEN) throw new IOException("block move failed due to access token error"); 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 bedc61ec308a..2f71f5ffc065 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 @@ -46,6 +46,7 @@ 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.InvalidMagicNumberException; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; @@ -273,6 +274,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]; @@ -350,6 +352,7 @@ public void readBlock(final ExtendedBlock block, 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. " + @@ -518,6 +521,7 @@ 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 " + @@ -528,8 +532,9 @@ public void writeBlock(final ExtendedBlock block, } } catch (IOException e) { + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Exception transferring block"); if (isClient) { - BlockOpResponseProto.newBuilder() + XTraceProtoUtils.newBlockOpResponseProtoBuilder() .setStatus(ERROR) // NB: Unconditionally using the xfer addr w/o hostname .setFirstBadLink(targets[0].getXferAddr()) @@ -563,7 +568,7 @@ public void writeBlock(final ExtendedBlock block, " forwarding connect ack to upstream firstbadlink is " + firstBadLink); } - BlockOpResponseProto.newBuilder() + XTraceProtoUtils.newBlockOpResponseProtoBuilder() .setStatus(mirrorInStatus) .setFirstBadLink(firstBadLink) .build() @@ -677,7 +682,7 @@ public void blockChecksum(final ExtendedBlock block, } //write reply - BlockOpResponseProto.newBuilder() + XTraceProtoUtils.newBlockOpResponseProtoBuilder() .setStatus(SUCCESS) .setChecksumResponse(OpBlockChecksumResponseProto.newBuilder() .setBytesPerCrc(bytesPerCRC) @@ -847,6 +852,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) { @@ -931,6 +937,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(); } @@ -943,7 +950,7 @@ private void writeSuccessWithChecksumInfo(BlockSender blockSender, .setChunkOffset(blockSender.getOffset()) .build(); - BlockOpResponseProto response = BlockOpResponseProto.newBuilder() + BlockOpResponseProto response = XTraceProtoUtils.newBlockOpResponseProtoBuilder() .setStatus(SUCCESS) .setReadOpChecksumInfo(ckInfo) .build(); @@ -975,6 +982,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/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto index 83685d186041..0b91a07ec6c7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto @@ -195,6 +195,7 @@ message BlockOpResponseProto { * read. */ optional uint32 shortCircuitAccessVersion = 6; + optional bytes xtrace = 7; // X-Trace metadata } /** @@ -203,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 { From 9bea5f9c3e109983d6a09d1a35aae76879f7e54f Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 11 Oct 2013 11:03:55 -0400 Subject: [PATCH 020/112] Fixed a context joining problem in the block receiver --- .../apache/hadoop/hdfs/server/datanode/BlockReceiver.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 453fdf4b4d9d..24f56ac6298a 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 @@ -722,6 +722,7 @@ void receiveBlock( if (responder != null) { try { responder.join(datanode.getDnConf().getXceiverStopTimeout()); + ((PacketResponder)responder.getRunnable()).joinXtraceContext(); if (responder.isAlive()) { String msg = "Join on responder thread " + responder + " timed out"; @@ -919,6 +920,9 @@ public synchronized void close() { } running = false; notifyAll(); + } + + public void joinXtraceContext() { XTraceContext.joinContext(xtrace); // rejoin the end xtrace context } @@ -1075,7 +1079,7 @@ public void run() { continue; } } - XTraceContext.logEvent(PacketReceiver.class, "PacketResponder", "Acknowledging packet", "seqno", seqno, "AckTimeNanos", totalAckTimeNanos); + XTraceContext.logEvent(PacketReceiver.class, "PacketResponder", "Acknowledging packet", "seqno", expected, "AckTimeNanos", totalAckTimeNanos); PipelineAck replyAck = new PipelineAck(expected, replies, totalAckTimeNanos); if (replyAck.isSuccess() && From ed479e2a58916c2f3892b4e7c9c93577f11ed710 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 11 Oct 2013 11:05:40 -0400 Subject: [PATCH 021/112] Instrumented start and end of many DFSClient api function calls --- .../org/apache/hadoop/hdfs/DFSClient.java | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 19f867adf500..55b01252da07 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -163,6 +163,8 @@ import com.google.common.base.Preconditions; import com.google.common.net.InetAddresses; +import edu.berkeley.xtrace.XTraceContext; + /******************************************************** * DFSClient can connect to a Hadoop Filesystem and * perform basic file tasks. It uses the ClientProtocol @@ -1018,12 +1020,15 @@ public LocatedBlocks getLocatedBlocks(String src, long start, long length) static LocatedBlocks callGetBlockLocations(ClientProtocol namenode, String src, long start, long length) throws IOException { + XTraceContext.startTrace("DFSClient", "GetBlockLocations", "getBlockLocations", src+" ["+start+":"+(start+length)+"]"); try { return namenode.getBlockLocations(src, start, length); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, FileNotFoundException.class, UnresolvedPathException.class); + } finally { + XTraceContext.logEvent("DFSClient", "GetBlockLocations complete"); } } @@ -1166,6 +1171,7 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum, public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) throws IOException, UnresolvedLinkException { checkOpen(); + XTraceContext.startTrace("DFSClient", "Open", "open", src); // Get block info from namenode return new DFSInputStream(this, src, buffersize, verifyChecksum); } @@ -1337,6 +1343,7 @@ public DFSOutputStream create(String src, + favoredNodes[i].getPort(); } } + XTraceContext.startTrace("DFSClient", "Create", "create", src); final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this, src, masked, flag, createParent, replication, blockSize, progress, buffersize, dfsClientConf.createChecksum(checksumOpt), favoredNodeStrs); @@ -1359,6 +1366,7 @@ private DFSOutputStream primitiveAppend(String src, EnumSet flag, } return null; } + XTraceContext.startTrace("DFSClient", "Append", "append", src); return callAppend(stat, src, buffersize, progress); } return null; @@ -1383,6 +1391,7 @@ public DFSOutputStream primitiveCreate(String src, CreateFlag.validate(flag); DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress); if (result == null) { + XTraceContext.startTrace("DFSClient", "Create", "create", src); DataChecksum checksum = dfsClientConf.createChecksum(checksumOpt); result = DFSOutputStream.newStreamForCreate(this, src, absPermission, flag, createParent, replication, blockSize, progress, buffersize, @@ -1463,6 +1472,7 @@ private DFSOutputStream callAppend(HdfsFileStatus stat, String src, public HdfsDataOutputStream append(final String src, final int buffersize, final Progressable progress, final FileSystem.Statistics statistics ) throws IOException { + XTraceContext.startTrace("DFSClient", "Append", "append", src); final DFSOutputStream out = append(src, buffersize, progress); return new HdfsDataOutputStream(out, statistics, out.getInitialLen()); } @@ -1510,6 +1520,7 @@ public boolean setReplication(String src, short replication) public boolean rename(String src, String dst) throws IOException { checkOpen(); try { + XTraceContext.startTrace("DFSClient", "Rename", "rename", src+" -> "+dst); return namenode.rename(src, dst); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1517,6 +1528,8 @@ public boolean rename(String src, String dst) throws IOException { DSQuotaExceededException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); + } finally { + XTraceContext.logEvent("DFSClient", "Rename complete"); } } @@ -1527,11 +1540,14 @@ public boolean rename(String src, String dst) throws IOException { public void concat(String trg, String [] srcs) throws IOException { checkOpen(); try { + XTraceContext.startTrace("DFSClient", "Concat", "concat"); namenode.concat(trg, srcs); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); + } finally { + XTraceContext.logEvent("DFSClient", "Concat complete"); } } /** @@ -1542,6 +1558,7 @@ public void rename(String src, String dst, Options.Rename... options) throws IOException { checkOpen(); try { + XTraceContext.startTrace("DFSClient", "Rename", "rename", src+" -> "+dst); namenode.rename2(src, dst, options); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1553,6 +1570,8 @@ public void rename(String src, String dst, Options.Rename... options) NSQuotaExceededException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); + } finally { + XTraceContext.logEvent("DFSClient", "Rename complete"); } } /** @@ -1562,7 +1581,12 @@ public void rename(String src, String dst, Options.Rename... options) @Deprecated public boolean delete(String src) throws IOException { checkOpen(); + XTraceContext.startTrace("DFSClient", "Delete", "delete", src); + try { // xtrace try return namenode.delete(src, true); + } finally { + XTraceContext.logEvent("DFSClient", "Delete complete"); + } } /** @@ -1575,6 +1599,7 @@ public boolean delete(String src) throws IOException { public boolean delete(String src, boolean recursive) throws IOException { checkOpen(); try { + XTraceContext.startTrace("DFSClient", "Delete", "delete", src); return namenode.delete(src, recursive); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1582,6 +1607,8 @@ public boolean delete(String src, boolean recursive) throws IOException { SafeModeException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); + } finally { + XTraceContext.logEvent("DFSClient", "Delete complete"); } } @@ -1615,11 +1642,14 @@ public DirectoryListing listPaths(String src, byte[] startAfter, throws IOException { checkOpen(); try { + XTraceContext.startTrace("DFSClient", "ListPaths", "listPaths", src); return namenode.getListing(src, startAfter, needLocation); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, FileNotFoundException.class, UnresolvedPathException.class); + } finally { + XTraceContext.logEvent("DFSClient", "ListPaths complete"); } } @@ -1634,11 +1664,14 @@ public DirectoryListing listPaths(String src, byte[] startAfter, public HdfsFileStatus getFileInfo(String src) throws IOException { checkOpen(); try { + XTraceContext.startTrace("DFSClient", "GetFileInfo", "getFileInfo", src); return namenode.getFileInfo(src); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, FileNotFoundException.class, UnresolvedPathException.class); + } finally { + XTraceContext.logEvent("DFSClient", "GetFileInfo complete"); } } @@ -2117,9 +2150,12 @@ public String createSnapshot(String snapshotRoot, String snapshotName) throws IOException { checkOpen(); try { + XTraceContext.startTrace("DFSClient", "CreateSnapshot", "createSnapshot", snapshotRoot); return namenode.createSnapshot(snapshotRoot, snapshotName); } catch(RemoteException re) { throw re.unwrapRemoteException(); + } finally { + XTraceContext.logEvent("DFSClient", "CreateSnapshot complete"); } } @@ -2135,9 +2171,12 @@ public String createSnapshot(String snapshotRoot, String snapshotName) public void deleteSnapshot(String snapshotRoot, String snapshotName) throws IOException { try { + XTraceContext.startTrace("DFSClient", "DeleteSnapshot", "deleteSnapshot", snapshotRoot); namenode.deleteSnapshot(snapshotRoot, snapshotName); } catch(RemoteException re) { throw re.unwrapRemoteException(); + } finally { + XTraceContext.logEvent("DFSClient", "DeleteSnapshot complete"); } } @@ -2153,9 +2192,12 @@ public void renameSnapshot(String snapshotDir, String snapshotOldName, String snapshotNewName) throws IOException { checkOpen(); try { + XTraceContext.startTrace("DFSClient", "RenameSnapshot", "renameSnapshot", snapshotOldName+" -> "+snapshotNewName); namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName); } catch(RemoteException re) { throw re.unwrapRemoteException(); + } finally { + XTraceContext.logEvent("DFSClient", "RenameSnapshot complete"); } } @@ -2341,6 +2383,7 @@ public boolean primitiveMkdir(String src, FsPermission absPermission, boolean createParent) throws IOException { checkOpen(); + XTraceContext.startTrace("DFSClient", "MkDir", "mkdir", src); if (absPermission == null) { absPermission = FsPermission.getDefault().applyUMask(dfsClientConf.uMask); @@ -2362,6 +2405,8 @@ public boolean primitiveMkdir(String src, FsPermission absPermission, DSQuotaExceededException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); + } finally { + XTraceContext.logEvent("DFSClient", "MkDir complete"); } } From 327e39044599b082bc3876cedf06d5a9ff287d27 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 11 Oct 2013 22:02:43 -0400 Subject: [PATCH 022/112] Added some resource tracing events --- .../src/main/java/org/apache/hadoop/ipc/Client.java | 7 ++++--- .../java/org/apache/hadoop/hdfs/DFSOutputStream.java | 8 ++++++++ .../hadoop/hdfs/server/datanode/BlockReceiver.java | 10 ++++++++++ 3 files changed, 22 insertions(+), 3 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index 8b9b9e87c7ad..315bc3188a18 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -100,6 +100,7 @@ import edu.berkeley.xtrace.XTraceMetadata; import edu.berkeley.xtrace.XTraceMetadataCollection; import edu.berkeley.xtrace.XTraceProcess; +import edu.berkeley.xtrace.XTraceResourceTracing; /** A client for an IPC service. IPC calls take a single {@link Writable} as a * parameter, and return a {@link Writable} as their value. A service runs on @@ -1352,7 +1353,7 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, LOG.warn("interrupted waiting to send rpc request to server", e); throw new IOException(e); } - + XTraceResourceTracing.waitStart(); boolean interrupted = false; synchronized (call) { while (!call.done) { @@ -1368,10 +1369,10 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, // set the interrupt flag now that we are done waiting Thread.currentThread().interrupt(); } - + XTraceResourceTracing.waitEnd(); if (call.xtrace != null) { - XTraceContext.setThreadContext(call.xtrace); + XTraceContext.joinContext(call.xtrace); } if (call.error != null) { if (call.error instanceof RemoteException) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index fcec7fdde93d..1f4fcff601f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -95,6 +95,7 @@ import edu.berkeley.xtrace.XTraceContext; import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceResourceTracing; /**************************************************************** @@ -1533,7 +1534,12 @@ private void queueCurrentPacket() { private void waitAndQueueCurrentPacket() throws IOException { synchronized (dataQueue) { // If queue is full, then wait till we have enough space + boolean loggedWait = false; while (!closed && dataQueue.size() + ackQueue.size() > MAX_PACKETS) { + if (!loggedWait) { + XTraceResourceTracing.waitStart(); + loggedWait = true; + } try { XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Output queues full, waiting for space..."); dataQueue.wait(); @@ -1549,6 +1555,8 @@ private void waitAndQueueCurrentPacket() throws IOException { break; } } + if (loggedWait) + XTraceResourceTracing.waitEnd(); checkClosed(); queueCurrentPacket(); } 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 24f56ac6298a..13d6ddb8920e 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 @@ -56,6 +56,7 @@ import edu.berkeley.xtrace.XTraceContext; import edu.berkeley.xtrace.XTraceMetadata; +import edu.berkeley.xtrace.XTraceResourceTracing; /** A class that receives a block and writes to its own disk, meanwhile * may copies it to another site. If a throttler is provided, @@ -907,6 +908,7 @@ synchronized void enqueue(final long seqno, */ @Override public synchronized void close() { + XTraceResourceTracing.waitStart(); while (running && ackQueue.size() != 0 && datanode.shouldRun) { try { wait(); @@ -920,6 +922,7 @@ public synchronized void close() { } running = false; notifyAll(); + XTraceResourceTracing.waitEnd(); } public void joinXtraceContext() { @@ -964,13 +967,20 @@ public void run() { if (seqno != PipelineAck.UNKOWN_SEQNO || type == PacketResponderType.LAST_IN_PIPELINE) { synchronized (this) { + boolean loggedWait = false; while (running && datanode.shouldRun && ackQueue.size() == 0) { + if (!loggedWait) { + XTraceResourceTracing.waitStart(); + loggedWait = true; + } if (LOG.isDebugEnabled()) { LOG.debug(myString + ": seqno=" + seqno + " waiting for local datanode to finish write."); } wait(); } + if (loggedWait) + XTraceResourceTracing.waitEnd(); if (!running || !datanode.shouldRun) { break; } From 1c8aa47970c89b4fa114e5d95becbbb64d2c036b Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 16 Oct 2013 14:59:57 -0400 Subject: [PATCH 023/112] Added an extra log statement to the DFSOutputStream --- .../src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 1f4fcff601f3..580225d36d63 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -448,6 +448,7 @@ private void setFavoredNodes(String[] favoredNodes) { private void initDataStreaming() { this.setName("DataStreamer for file " + src + " block " + block); + XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Kicking off response processor"); response = new ResponseProcessor(nodes); response.start(); stage = BlockConstructionStage.DATA_STREAMING; From 198eb9e36d14425117dc3ed3da89af03d4f2c75c Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 16 Oct 2013 17:50:48 -0400 Subject: [PATCH 024/112] Added log statement to indicate that we're forcing a namenode block update --- .../org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 75f42f959d76..b931daab17c0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -58,6 +58,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Maps; +import edu.berkeley.xtrace.XTraceContext; + /** * A thread per active or standby namenode to perform: *
    @@ -313,6 +315,7 @@ void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) { pendingIncrementalBR.put( bInfo.getBlock().getBlockId(), bInfo); pendingReceivedRequests++; + XTraceContext.logEvent(BPServiceActor.class, "BPServiceActor", "Triggering block report to namenode"); pendingIncrementalBR.notifyAll(); } } From c93495b7d13f14ac21909c13a29b623d301a4b9f Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 16 Oct 2013 18:04:48 -0400 Subject: [PATCH 025/112] Added some instrumentation of some of the main locks used on the namenode. --- .../server/blockmanagement/Host2NodesMap.java | 44 ++++++++++++++----- .../hdfs/server/namenode/FSDirectory.java | 16 +++++++ .../hdfs/server/namenode/FSNamesystem.java | 10 +++++ 3 files changed, 60 insertions(+), 10 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java index 6f9049a960c1..0972f9f91d91 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java @@ -27,6 +27,8 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.DFSUtil; +import edu.berkeley.xtrace.XTraceResourceTracing; + /** A map from host names to datanode descriptors. */ @InterfaceAudience.Private @InterfaceStability.Evolving @@ -34,6 +36,28 @@ class Host2NodesMap { private HashMap map = new HashMap(); private ReadWriteLock hostmapLock = new ReentrantReadWriteLock(); + + private void lockReadLock() { + XTraceResourceTracing.requestLock(hostmapLock.readLock()); + hostmapLock.readLock().lock(); + XTraceResourceTracing.acquiredLock(hostmapLock.readLock()); + } + + private void unlockReadLock() { + hostmapLock.readLock().unlock(); + XTraceResourceTracing.releasedLock(hostmapLock.readLock()); + } + + private void lockWriteLock() { + XTraceResourceTracing.requestLock(hostmapLock.writeLock()); + hostmapLock.writeLock().lock(); + XTraceResourceTracing.acquiredLock(hostmapLock.writeLock()); + } + + private void unlockWriteLock() { + hostmapLock.writeLock().unlock(); + XTraceResourceTracing.releasedLock(hostmapLock.writeLock()); + } /** Check if node is already in the map. */ boolean contains(DatanodeDescriptor node) { @@ -42,7 +66,7 @@ boolean contains(DatanodeDescriptor node) { } String ipAddr = node.getIpAddr(); - hostmapLock.readLock().lock(); + lockReadLock(); try { DatanodeDescriptor[] nodes = map.get(ipAddr); if (nodes != null) { @@ -53,7 +77,7 @@ boolean contains(DatanodeDescriptor node) { } } } finally { - hostmapLock.readLock().unlock(); + unlockReadLock(); } return false; } @@ -62,7 +86,7 @@ boolean contains(DatanodeDescriptor node) { * return true if the node is added; false otherwise. */ boolean add(DatanodeDescriptor node) { - hostmapLock.writeLock().lock(); + lockWriteLock(); try { if (node==null || contains(node)) { return false; @@ -82,7 +106,7 @@ boolean add(DatanodeDescriptor node) { map.put(ipAddr, newNodes); return true; } finally { - hostmapLock.writeLock().unlock(); + unlockWriteLock(); } } @@ -95,7 +119,7 @@ boolean remove(DatanodeDescriptor node) { } String ipAddr = node.getIpAddr(); - hostmapLock.writeLock().lock(); + lockWriteLock(); try { DatanodeDescriptor[] nodes = map.get(ipAddr); @@ -128,7 +152,7 @@ boolean remove(DatanodeDescriptor node) { return true; } } finally { - hostmapLock.writeLock().unlock(); + unlockWriteLock(); } } @@ -141,7 +165,7 @@ DatanodeDescriptor getDatanodeByHost(String ipAddr) { return null; } - hostmapLock.readLock().lock(); + lockReadLock(); try { DatanodeDescriptor[] nodes = map.get(ipAddr); // no entry @@ -155,7 +179,7 @@ DatanodeDescriptor getDatanodeByHost(String ipAddr) { // more than one node return nodes[DFSUtil.getRandom().nextInt(nodes.length)]; } finally { - hostmapLock.readLock().unlock(); + unlockReadLock(); } } @@ -170,7 +194,7 @@ public DatanodeDescriptor getDatanodeByXferAddr(String ipAddr, return null; } - hostmapLock.readLock().lock(); + lockReadLock(); try { DatanodeDescriptor[] nodes = map.get(ipAddr); // no entry @@ -184,7 +208,7 @@ public DatanodeDescriptor getDatanodeByXferAddr(String ipAddr, } return null; } finally { - hostmapLock.readLock().unlock(); + unlockReadLock(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index e4bf91c3c4fb..428acce1e005 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -74,6 +74,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import edu.berkeley.xtrace.XTraceResourceTracing; + /************************************************* * FSDirectory stores the filesystem directory state. * It handles writing/loading values to disk, and logging @@ -119,19 +121,26 @@ private static INodeDirectoryWithQuota createRoot(FSNamesystem namesystem) { // utility methods to acquire and release read lock and write lock void readLock() { + XTraceResourceTracing.requestLock(this.dirLock.readLock(), "dirLock readLock"); this.dirLock.readLock().lock(); + XTraceResourceTracing.acquiredLock(this.dirLock.readLock()); } void readUnlock() { this.dirLock.readLock().unlock(); + XTraceResourceTracing.releasedLock(this.dirLock.readLock()); } void writeLock() { + + XTraceResourceTracing.requestLock(this.dirLock.writeLock(), "dirLock writeLock"); this.dirLock.writeLock().lock(); + XTraceResourceTracing.acquiredLock(this.dirLock.writeLock()); } void writeUnlock() { this.dirLock.writeLock().unlock(); + XTraceResourceTracing.releasedLock(this.dirLock.writeLock()); } boolean hasWriteLock() { @@ -239,9 +248,14 @@ public void close() throws IOException { */ void waitForReady() { if (!ready) { + boolean waited = false; writeLock(); try { while (!ready) { + if (!waited) { + waited = true; + XTraceResourceTracing.waitStart(); + } try { cond.await(5000, TimeUnit.MILLISECONDS); } catch (InterruptedException ie) { @@ -250,6 +264,8 @@ void waitForReady() { } finally { writeUnlock(); } + if (waited) + XTraceResourceTracing.waitEnd(); } } 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..38c56398dada 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 @@ -231,6 +231,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import edu.berkeley.xtrace.XTraceResourceTracing; + /*************************************************** * FSNamesystem does the actual bookkeeping work for the * DataNode. @@ -1144,23 +1146,31 @@ public static List getSharedEditsDirs(Configuration conf) { @Override public void readLock() { + XTraceResourceTracing.requestLock(this.fsLock.readLock(), "FSNamesystem readLock"); this.fsLock.readLock().lock(); + XTraceResourceTracing.acquiredLock(this.fsLock.readLock()); } @Override public void readUnlock() { this.fsLock.readLock().unlock(); + XTraceResourceTracing.releasedLock(this.fsLock.readLock()); } @Override public void writeLock() { + XTraceResourceTracing.requestLock(this.fsLock.writeLock(), "FSNamesystem writeLock"); this.fsLock.writeLock().lock(); + XTraceResourceTracing.acquiredLock(this.fsLock.writeLock()); } @Override public void writeLockInterruptibly() throws InterruptedException { + XTraceResourceTracing.requestLock(this.fsLock.writeLock(), "FSNamesystem writeLock"); this.fsLock.writeLock().lockInterruptibly(); + XTraceResourceTracing.acquiredLock(this.fsLock.writeLock()); } @Override public void writeUnlock() { this.fsLock.writeLock().unlock(); + XTraceResourceTracing.releasedLock(this.fsLock.writeLock()); } @Override public boolean hasWriteLock() { From 6307cc547144cb875cc54d269b10b7cd327c33d6 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 16 Oct 2013 18:22:42 -0400 Subject: [PATCH 026/112] Removed some annoying unnecessary RPC log messages, and added a 'name' tag to name the spans in the rpc invoker. --- .../main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java | 2 +- .../src/main/java/org/apache/hadoop/ipc/Server.java | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java index d4b6f3b3686c..97d338ec5883 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java @@ -581,7 +581,7 @@ public Writable call(RPC.Server server, String protocol, if (server.verbose) LOG.info("Call: protocol=" + protocol + ", method=" + methodName); - XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Invoking method "+methodName, "Protocol", protocol); + XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Invoking method "+methodName, "Protocol", protocol, "Name", methodName); Collection start_context = XTraceContext.getThreadContext(); try { // xtrace try diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index c13f85f18cc1..b9ce5fcba356 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -2036,8 +2036,6 @@ public void run() { final Call call = callQueue.take(); // pop the queue; maybe blocked here if (call.xtrace != null) { XTraceContext.setThreadContext(call.xtrace); - XTraceContext.logEvent(RPC.class, "RPC Server", "Processing RPC call", "From", call.connection, - "Protocol", call.connection.protocolName, "rpcKind", call.rpcKind); } if (LOG.isDebugEnabled()) { LOG.debug(getName() + ": " + call + " for RpcKind " + call.rpcKind); @@ -2317,8 +2315,6 @@ private void setupResponse(ByteArrayOutputStream responseBuf, headerBuilder.setStatus(status); headerBuilder.setServerIpcVersionNum(CURRENT_VERSION); if (XTraceContext.isValid()) { - XTraceContext.logEvent(RPC.class, "RPC Server", "Sending " + status + " response", - "Connection", call.connection, "Call ID", call.callId); headerBuilder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); } /* X-Trace: we have to send in the response the last event in the server From 84e597d778c28dc5c261341690fa4deb84c2e332 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 17 Oct 2013 10:54:50 -0400 Subject: [PATCH 027/112] Changed the startTrace logpoints in DataXceiver to just logEvents. Starting traces here isn't such a good idea; background tasks end up accidentally starting traces when really we're not interestedin them. --- .../hadoop/hdfs/server/datanode/DataXceiver.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) 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 2f71f5ffc065..e2f720cd795a 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 @@ -246,7 +246,7 @@ public void run() { public void requestShortCircuitFds(final ExtendedBlock blk, final Token token, int maxVersion) throws IOException { - XTraceContext.startTrace("DataNode", "RequestShortCircuitFds"); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode RequestShortCircuitFds"); updateCurrentThreadName("Passing file descriptors for block " + blk); BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder(); @@ -308,7 +308,7 @@ public void readBlock(final ExtendedBlock block, final long blockOffset, final long length, final boolean sendChecksum) throws IOException { - XTraceContext.startTrace("DataNode", "ReadBlock"); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode ReadBlock"); previousOpClientName = clientName; OutputStream baseStream = getOutputStream(); @@ -406,7 +406,7 @@ public void writeBlock(final ExtendedBlock block, final long maxBytesRcvd, final long latestGenerationStamp, DataChecksum requestedChecksum) throws IOException { - XTraceContext.startTrace("DataNode", "WriteBlock"); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode WriteBlock"); previousOpClientName = clientname; updateCurrentThreadName("Receiving block " + block); final boolean isDatanode = clientname.length() == 0; @@ -634,7 +634,7 @@ public void transferBlock(final ExtendedBlock blk, final Token blockToken, final String clientName, final DatanodeInfo[] targets) throws IOException { - XTraceContext.startTrace("DataNode", "TransferBlock"); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode TransferBlock"); checkAccess(socketOut, true, blk, blockToken, Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY); previousOpClientName = clientName; @@ -653,7 +653,7 @@ public void transferBlock(final ExtendedBlock blk, @Override public void blockChecksum(final ExtendedBlock block, final Token blockToken) throws IOException { - XTraceContext.startTrace("DataNode", "BlockChecksum"); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode BlockChecksum"); final DataOutputStream out = new DataOutputStream( getOutputStream()); checkAccess(out, true, block, blockToken, @@ -706,7 +706,7 @@ public void blockChecksum(final ExtendedBlock block, @Override public void copyBlock(final ExtendedBlock block, final Token blockToken) throws IOException { - XTraceContext.startTrace("DataNode", "CopyBlock"); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode CopyBlock"); updateCurrentThreadName("Copying block " + block); // Read in the header if (datanode.isBlockTokenEnabled) { @@ -782,7 +782,7 @@ public void replaceBlock(final ExtendedBlock block, final Token blockToken, final String delHint, final DatanodeInfo proxySource) throws IOException { - XTraceContext.startTrace("DataNode", "ReplaceBlock"); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode ReplaceBlock"); updateCurrentThreadName("Replacing block " + block + " from " + delHint); /* read header */ From bc4a09662a994b437050dd1871b86f009649c88a Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 17 Oct 2013 10:56:08 -0400 Subject: [PATCH 028/112] Use XTraceResourceTracing branching API to log the explicit computation boundaries when kicking off new threads. Also, add some names when setting XTraceContext to give spans names. --- .../java/org/apache/hadoop/ipc/ProtobufRpcEngine.java | 2 +- .../main/java/org/apache/hadoop/hdfs/DFSOutputStream.java | 8 ++++---- .../hadoop/hdfs/protocol/datatransfer/Receiver.java | 2 +- .../apache/hadoop/hdfs/server/datanode/BlockReceiver.java | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java index 97d338ec5883..96961ef243a8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java @@ -581,7 +581,7 @@ public Writable call(RPC.Server server, String protocol, if (server.verbose) LOG.info("Call: protocol=" + protocol + ", method=" + methodName); - XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Invoking method "+methodName, "Protocol", protocol, "Name", methodName); + XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Invoking method "+methodName, "Protocol", protocol, "Name", "RPC: "+methodName); Collection start_context = XTraceContext.getThreadContext(); try { // xtrace try diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 580225d36d63..eea9a9e9c61e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -371,7 +371,7 @@ public DatanodeInfo load(DatanodeInfo key) throws Exception { private final boolean isAppend; /** XTrace Context when streamer is created */ - private Collection xtrace = XTraceContext.getThreadContext(); + private Collection xtrace = XTraceResourceTracing.getContextForNewThread(); /** * Default construction for file create @@ -472,7 +472,7 @@ private void endBlock() { */ @Override public void run() { - XTraceContext.joinContext(xtrace); + XTraceContext.setThreadContext(xtrace, "Client DataStreamer Thread"); try { //xtrace try long lastPacket = Time.now(); @@ -748,7 +748,7 @@ private class ResponseProcessor extends Daemon { private DatanodeInfo[] targets = null; private boolean isLastPacketInBlock = false; - private Collection xtrace = XTraceContext.getThreadContext(); + private Collection xtrace = XTraceResourceTracing.getContextForNewThread(); ResponseProcessor (DatanodeInfo[] targets) { this.targets = targets; @@ -756,7 +756,7 @@ private class ResponseProcessor extends Daemon { @Override public void run() { - XTraceContext.joinContext(xtrace); + XTraceContext.setThreadContext(xtrace, "Client Data Streamer Response Processor Thread"); try { // xtrace try setName("ResponseProcessor for block " + block); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index a950b177e24e..8f4367c8b47e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -94,7 +94,7 @@ private static void XTraceJoinStart(String type, BaseHeaderProto header) { ByteString xbs = header.getXtrace(); XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); if (xmd!=null && xmd.isValid()) - XTraceContext.setThreadContext(xmd); + XTraceContext.setThreadContext(xmd, "DataNode Receiver Thread: "+type+" Request"); } } 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 13d6ddb8920e..17c62ea4d915 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 @@ -857,7 +857,7 @@ class PacketResponder implements Runnable, Closeable { private final String myString; /** XTrace context when packet responder was created */ - private Collection xtrace = XTraceContext.getThreadContext(); + private Collection xtrace = XTraceResourceTracing.getContextForNewThread(); @Override public String toString() { @@ -935,7 +935,7 @@ public void joinXtraceContext() { */ @Override public void run() { - XTraceContext.setThreadContext(xtrace); // set the start xtrace context + XTraceContext.setThreadContext(xtrace, "DataNode Packet Responder Thread"); // set the start xtrace context try { // xtrace try boolean lastPacketInBlock = false; From 589c555615533c6967dda76d5cc9d03628fd44d3 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 18 Oct 2013 09:55:30 -0400 Subject: [PATCH 029/112] Slightly modify server to extend the boundary of where the xtrace context is cleared. --- .../src/main/java/org/apache/hadoop/ipc/Server.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index b9ce5fcba356..31656658c68f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -503,8 +503,6 @@ public Call(int id, int retryCount, Writable param, Connection connection, XTraceContext.logEvent(RPC.class, "RPC Server", "Received RPC call", "CallID", id); this.xtrace = XTraceContext.getThreadContext(); } - XTraceContext.clearThreadContext(); //nothing happens until call gets dequeued - //we use call.xtrace to keep track } @Override @@ -1810,6 +1808,8 @@ private void processOneRpc(byte[] buf) ioe.getClass().getName(), ioe.getMessage()); responder.doRespond(call); throw wrse; + } finally { + XTraceContext.clearThreadContext(); } } From ea700a55b9ef8448a73c898f317c01a9e637a1d6 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 21 Oct 2013 11:30:24 -0400 Subject: [PATCH 030/112] Removed all resource tracing stuff, which will be moved to a new branch. --- hadoop-common-project/hadoop-common/pom.xml | 5 --- .../java/org/apache/hadoop/ipc/Client.java | 4 --- .../apache/hadoop/hdfs/DFSOutputStream.java | 15 +-------- .../hdfs/protocol/datatransfer/Receiver.java | 2 +- .../server/blockmanagement/Host2NodesMap.java | 7 ---- .../hdfs/server/datanode/BlockReceiver.java | 15 +-------- .../hdfs/server/namenode/FSDirectory.java | 16 ---------- .../hdfs/server/namenode/FSNamesystem.java | 10 +----- .../org/apache/hadoop/examples/WordCount.java | 2 -- hadoop-project-dist/pom.xml | 32 +++++++++++++++++++ 10 files changed, 36 insertions(+), 72 deletions(-) diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml index de176400321e..f264b48adff2 100644 --- a/hadoop-common-project/hadoop-common/pom.xml +++ b/hadoop-common-project/hadoop-common/pom.xml @@ -254,11 +254,6 @@ commons-compress 1.4 - - edu.berkeley.xtrace - xtrace - 2.1-20120824 - diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index 315bc3188a18..df895fee5d48 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -99,8 +99,6 @@ import edu.berkeley.xtrace.XTraceContext; import edu.berkeley.xtrace.XTraceMetadata; import edu.berkeley.xtrace.XTraceMetadataCollection; -import edu.berkeley.xtrace.XTraceProcess; -import edu.berkeley.xtrace.XTraceResourceTracing; /** A client for an IPC service. IPC calls take a single {@link Writable} as a * parameter, and return a {@link Writable} as their value. A service runs on @@ -1353,7 +1351,6 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, LOG.warn("interrupted waiting to send rpc request to server", e); throw new IOException(e); } - XTraceResourceTracing.waitStart(); boolean interrupted = false; synchronized (call) { while (!call.done) { @@ -1369,7 +1366,6 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, // set the interrupt flag now that we are done waiting Thread.currentThread().interrupt(); } - XTraceResourceTracing.waitEnd(); if (call.xtrace != null) { XTraceContext.joinContext(call.xtrace); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index eea9a9e9c61e..b091409a8945 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -95,7 +95,6 @@ import edu.berkeley.xtrace.XTraceContext; import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceResourceTracing; /**************************************************************** @@ -371,7 +370,7 @@ public DatanodeInfo load(DatanodeInfo key) throws Exception { private final boolean isAppend; /** XTrace Context when streamer is created */ - private Collection xtrace = XTraceResourceTracing.getContextForNewThread(); + private Collection xtrace = null; /** * Default construction for file create @@ -472,7 +471,6 @@ private void endBlock() { */ @Override public void run() { - XTraceContext.setThreadContext(xtrace, "Client DataStreamer Thread"); try { //xtrace try long lastPacket = Time.now(); @@ -747,8 +745,6 @@ private class ResponseProcessor extends Daemon { private volatile boolean responderClosed = false; private DatanodeInfo[] targets = null; private boolean isLastPacketInBlock = false; - - private Collection xtrace = XTraceResourceTracing.getContextForNewThread(); ResponseProcessor (DatanodeInfo[] targets) { this.targets = targets; @@ -756,7 +752,6 @@ private class ResponseProcessor extends Daemon { @Override public void run() { - XTraceContext.setThreadContext(xtrace, "Client Data Streamer Response Processor Thread"); try { // xtrace try setName("ResponseProcessor for block " + block); @@ -1535,14 +1530,8 @@ private void queueCurrentPacket() { private void waitAndQueueCurrentPacket() throws IOException { synchronized (dataQueue) { // If queue is full, then wait till we have enough space - boolean loggedWait = false; while (!closed && dataQueue.size() + ackQueue.size() > MAX_PACKETS) { - if (!loggedWait) { - XTraceResourceTracing.waitStart(); - loggedWait = true; - } try { - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Output queues full, waiting for space..."); dataQueue.wait(); } catch (InterruptedException e) { // If we get interrupted while waiting to queue data, we still need to get rid @@ -1556,8 +1545,6 @@ private void waitAndQueueCurrentPacket() throws IOException { break; } } - if (loggedWait) - XTraceResourceTracing.waitEnd(); checkClosed(); queueCurrentPacket(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index 8f4367c8b47e..a950b177e24e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -94,7 +94,7 @@ private static void XTraceJoinStart(String type, BaseHeaderProto header) { ByteString xbs = header.getXtrace(); XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); if (xmd!=null && xmd.isValid()) - XTraceContext.setThreadContext(xmd, "DataNode Receiver Thread: "+type+" Request"); + XTraceContext.setThreadContext(xmd); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java index 0972f9f91d91..1170de0f562f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java @@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.DFSUtil; -import edu.berkeley.xtrace.XTraceResourceTracing; /** A map from host names to datanode descriptors. */ @InterfaceAudience.Private @@ -38,25 +37,19 @@ class Host2NodesMap { private ReadWriteLock hostmapLock = new ReentrantReadWriteLock(); private void lockReadLock() { - XTraceResourceTracing.requestLock(hostmapLock.readLock()); hostmapLock.readLock().lock(); - XTraceResourceTracing.acquiredLock(hostmapLock.readLock()); } private void unlockReadLock() { hostmapLock.readLock().unlock(); - XTraceResourceTracing.releasedLock(hostmapLock.readLock()); } private void lockWriteLock() { - XTraceResourceTracing.requestLock(hostmapLock.writeLock()); hostmapLock.writeLock().lock(); - XTraceResourceTracing.acquiredLock(hostmapLock.writeLock()); } private void unlockWriteLock() { hostmapLock.writeLock().unlock(); - XTraceResourceTracing.releasedLock(hostmapLock.writeLock()); } /** Check if node is already in the map. */ 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 17c62ea4d915..6f9425adc512 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 @@ -56,7 +56,6 @@ import edu.berkeley.xtrace.XTraceContext; import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceResourceTracing; /** A class that receives a block and writes to its own disk, meanwhile * may copies it to another site. If a throttler is provided, @@ -856,8 +855,7 @@ class PacketResponder implements Runnable, Closeable { /** for log and error messages */ private final String myString; - /** XTrace context when packet responder was created */ - private Collection xtrace = XTraceResourceTracing.getContextForNewThread(); + private Collection xtrace = null; @Override public String toString() { @@ -908,7 +906,6 @@ synchronized void enqueue(final long seqno, */ @Override public synchronized void close() { - XTraceResourceTracing.waitStart(); while (running && ackQueue.size() != 0 && datanode.shouldRun) { try { wait(); @@ -922,7 +919,6 @@ public synchronized void close() { } running = false; notifyAll(); - XTraceResourceTracing.waitEnd(); } public void joinXtraceContext() { @@ -935,7 +931,6 @@ public void joinXtraceContext() { */ @Override public void run() { - XTraceContext.setThreadContext(xtrace, "DataNode Packet Responder Thread"); // set the start xtrace context try { // xtrace try boolean lastPacketInBlock = false; @@ -967,20 +962,13 @@ public void run() { if (seqno != PipelineAck.UNKOWN_SEQNO || type == PacketResponderType.LAST_IN_PIPELINE) { synchronized (this) { - boolean loggedWait = false; while (running && datanode.shouldRun && ackQueue.size() == 0) { - if (!loggedWait) { - XTraceResourceTracing.waitStart(); - loggedWait = true; - } if (LOG.isDebugEnabled()) { LOG.debug(myString + ": seqno=" + seqno + " waiting for local datanode to finish write."); } wait(); } - if (loggedWait) - XTraceResourceTracing.waitEnd(); if (!running || !datanode.shouldRun) { break; } @@ -1141,7 +1129,6 @@ public void run() { } } finally { // xtrace finally xtrace = XTraceContext.getThreadContext(); - XTraceContext.clearThreadContext(); } LOG.info(myString + " terminating"); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index 428acce1e005..e4bf91c3c4fb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -74,8 +74,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import edu.berkeley.xtrace.XTraceResourceTracing; - /************************************************* * FSDirectory stores the filesystem directory state. * It handles writing/loading values to disk, and logging @@ -121,26 +119,19 @@ private static INodeDirectoryWithQuota createRoot(FSNamesystem namesystem) { // utility methods to acquire and release read lock and write lock void readLock() { - XTraceResourceTracing.requestLock(this.dirLock.readLock(), "dirLock readLock"); this.dirLock.readLock().lock(); - XTraceResourceTracing.acquiredLock(this.dirLock.readLock()); } void readUnlock() { this.dirLock.readLock().unlock(); - XTraceResourceTracing.releasedLock(this.dirLock.readLock()); } void writeLock() { - - XTraceResourceTracing.requestLock(this.dirLock.writeLock(), "dirLock writeLock"); this.dirLock.writeLock().lock(); - XTraceResourceTracing.acquiredLock(this.dirLock.writeLock()); } void writeUnlock() { this.dirLock.writeLock().unlock(); - XTraceResourceTracing.releasedLock(this.dirLock.writeLock()); } boolean hasWriteLock() { @@ -248,14 +239,9 @@ public void close() throws IOException { */ void waitForReady() { if (!ready) { - boolean waited = false; writeLock(); try { while (!ready) { - if (!waited) { - waited = true; - XTraceResourceTracing.waitStart(); - } try { cond.await(5000, TimeUnit.MILLISECONDS); } catch (InterruptedException ie) { @@ -264,8 +250,6 @@ void waitForReady() { } finally { writeUnlock(); } - if (waited) - XTraceResourceTracing.waitEnd(); } } 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 38c56398dada..a3f3fcf6ced5 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 @@ -231,7 +231,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import edu.berkeley.xtrace.XTraceResourceTracing; + /*************************************************** * FSNamesystem does the actual bookkeeping work for the @@ -1146,31 +1146,23 @@ public static List getSharedEditsDirs(Configuration conf) { @Override public void readLock() { - XTraceResourceTracing.requestLock(this.fsLock.readLock(), "FSNamesystem readLock"); this.fsLock.readLock().lock(); - XTraceResourceTracing.acquiredLock(this.fsLock.readLock()); } @Override public void readUnlock() { this.fsLock.readLock().unlock(); - XTraceResourceTracing.releasedLock(this.fsLock.readLock()); } @Override public void writeLock() { - XTraceResourceTracing.requestLock(this.fsLock.writeLock(), "FSNamesystem writeLock"); this.fsLock.writeLock().lock(); - XTraceResourceTracing.acquiredLock(this.fsLock.writeLock()); } @Override public void writeLockInterruptibly() throws InterruptedException { - XTraceResourceTracing.requestLock(this.fsLock.writeLock(), "FSNamesystem writeLock"); this.fsLock.writeLock().lockInterruptibly(); - XTraceResourceTracing.acquiredLock(this.fsLock.writeLock()); } @Override public void writeUnlock() { this.fsLock.writeLock().unlock(); - XTraceResourceTracing.releasedLock(this.fsLock.writeLock()); } @Override public boolean hasWriteLock() { 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 8f3999b3cebc..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 @@ -31,8 +31,6 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.GenericOptionsParser; -import edu.berkeley.xtrace.XTraceContext; - public class WordCount { public static class TokenizerMapper diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml index 788cbdd9644c..e709e0aa35bd 100644 --- a/hadoop-project-dist/pom.xml +++ b/hadoop-project-dist/pom.xml @@ -48,10 +48,42 @@ hadoop-annotations provided + + + edu.berkeley.xtrace + xtrace + 2.1-20120824 + + + org.codehaus.mojo + aspectj-maven-plugin + 1.5 + + + + edu.brown.cs.systems + xtrace-resource-tracing + + + + + + + compile + + + 1.6 + 1.6 + 1.6 + synchronization + + + + org.apache.maven.plugins maven-jar-plugin From ba3446c0e6c5eb8b7040b08fcfdf721738a17532 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 21 Oct 2013 11:32:20 -0400 Subject: [PATCH 031/112] Added dependency to XResourceTracing --- hadoop-project-dist/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml index e709e0aa35bd..6026c6b58d9c 100644 --- a/hadoop-project-dist/pom.xml +++ b/hadoop-project-dist/pom.xml @@ -49,11 +49,21 @@ provided + + org.aspectj + aspectjrt + 1.7.3 + edu.berkeley.xtrace xtrace 2.1-20120824 + + edu.brown.cs.systems + xtrace-resource-tracing + 1.0 + From 9470d962525a7901b8fd5b4cc72a3b2462acc6eb Mon Sep 17 00:00:00 2001 From: unknown Date: Sat, 26 Oct 2013 08:28:22 -0700 Subject: [PATCH 032/112] Fixed an XTrace logging bug --- .../src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index b091409a8945..753b7abdd6ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -643,7 +643,7 @@ public void run() { } } catch (Throwable e) { DFSClient.LOG.warn("DataStreamer Exception", e); - XTraceContext.logEvent(DataStreamer.class, "DataStreamer Exception", "Message", e.getMessage()); + XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "DataStreamerException", "Message", e.getMessage()); if (e instanceof IOException) { setLastException((IOException)e); } From 7b09e0973bb3e56ebac9b3432d4f51fbc6a2e973 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 29 Oct 2013 11:26:32 -0400 Subject: [PATCH 033/112] Temporarily disabled code to propagate metadata between replicas, as it seems to be causing problems --- .../protocol/datatransfer/PacketReceiver.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java index 754b4d22739c..25988685e4aa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java @@ -230,15 +230,15 @@ public void mirrorPacketTo(DataOutputStream mirrorOut) throws IOException { * This updates the XTrace metadata in the packet header to the current context */ private void updateHeaderXTrace() { - // Only update context if there was a previous one, and we assume they have the exact - // same length, so we can just drop in a new packet header. - if (XTraceContext.isValid() && curHeader.hasXTraceContext()) { - PacketHeader newHeader = new PacketHeader(curHeader.getPacketLen(), curHeader.getOffsetInBlock(), - curHeader.getSeqno(), curHeader.isLastPacketInBlock(), curHeader.getDataLen(), - curHeader.getSyncBlock()); - curPacketBuf.position(0); - newHeader.putInBuffer(curPacketBuf); - } +// // Only update context if there was a previous one, and we assume they have the exact +// // same length, so we can just drop in a new packet header. +// if (XTraceContext.isValid() && curHeader.hasXTraceContext()) { +// PacketHeader newHeader = new PacketHeader(curHeader.getPacketLen(), curHeader.getOffsetInBlock(), +// curHeader.getSeqno(), curHeader.isLastPacketInBlock(), curHeader.getDataLen(), +// curHeader.getSyncBlock()); +// curPacketBuf.position(0); +// newHeader.putInBuffer(curPacketBuf); +// } } From 810dc4aa2f86b39db39b58ab2eba3ad06b93ca8b Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 29 Oct 2013 18:54:17 -0400 Subject: [PATCH 034/112] Fix for the mysterious bug that was causing checksum inconsistencies. Root cause was a bug in HDFS source code unrelated to X-Trace instrumentation. Have fixed the bug. --- .../protocol/datatransfer/PacketReceiver.java | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java index 25988685e4aa..b64e0507570c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java @@ -185,7 +185,7 @@ private void doRead(ReadableByteChannel ch, InputStream in) if (curHeader == null) { curHeader = new PacketHeader(); } - curHeader.setFieldsFromData(dataPlusChecksumLen, headerBuf); + curHeader.setFieldsFromData(payloadLen, headerBuf); curHeader.joinXTraceContext(); XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Finished reading packet"); @@ -230,15 +230,20 @@ public void mirrorPacketTo(DataOutputStream mirrorOut) throws IOException { * This updates the XTrace metadata in the packet header to the current context */ private void updateHeaderXTrace() { -// // Only update context if there was a previous one, and we assume they have the exact -// // same length, so we can just drop in a new packet header. -// if (XTraceContext.isValid() && curHeader.hasXTraceContext()) { -// PacketHeader newHeader = new PacketHeader(curHeader.getPacketLen(), curHeader.getOffsetInBlock(), -// curHeader.getSeqno(), curHeader.isLastPacketInBlock(), curHeader.getDataLen(), -// curHeader.getSyncBlock()); -// curPacketBuf.position(0); -// newHeader.putInBuffer(curPacketBuf); -// } + // Only update context if there was a previous one, and we assume they have the exact + // same length, so we can just drop in a new packet header. + if (XTraceContext.isValid() && curHeader.hasXTraceContext()) { + PacketHeader newHeader = new PacketHeader(curHeader.getPacketLen(), curHeader.getOffsetInBlock(), + curHeader.getSeqno(), curHeader.isLastPacketInBlock(), curHeader.getDataLen(), + curHeader.getSyncBlock()); + int priorPosition = curPacketBuf.position(); + int priorLimit = curPacketBuf.limit(); + curPacketBuf.position(0); + curPacketBuf.limit(newHeader.getSerializedSize()); + newHeader.putInBuffer(curPacketBuf); + curPacketBuf.position(priorPosition); + curPacketBuf.limit(priorLimit); + } } From 762bae6fc8ca260c8aca3c985c814128e77b3b17 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 29 Oct 2013 18:54:55 -0400 Subject: [PATCH 035/112] Some small instrumentation tweaks --- .../src/main/java/org/apache/hadoop/ipc/Client.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index df895fee5d48..c999c9057131 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -895,10 +895,11 @@ private synchronized void sendPing() throws IOException { @Override public void run() { + XTraceContext.logEvent(this.getClass(), "Client Connection", "RPC Response reader thread started"); if (LOG.isDebugEnabled()) LOG.debug(getName() + ": starting, having connections " + connections.size()); - + XTraceContext.clearThreadContext(); try { while (waitForWork()) {//wait here for work - read or close connection receiveRpcResponse(); @@ -959,6 +960,7 @@ public void run() { if (LOG.isDebugEnabled()) LOG.debug(getName() + " sending #" + call.id); + XTraceContext.logEvent(Client.Connection.class, "Client senderFuture", "Sending call"); byte[] data = d.getData(); int totalLength = d.getLength(); From d01c343f37c65568aae9e9916ae29dd0ed753fb8 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 11 Nov 2013 21:06:52 -0500 Subject: [PATCH 036/112] Added XResourceTracing as a bootclasspath option --- .../hadoop-common/src/main/bin/hadoop-config.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh index e5c40fcd6cf8..e409ac82f609 100644 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh @@ -293,3 +293,7 @@ if [ "$HADOOP_CLASSPATH" != "" ]; then fi fi +HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/xtrace-resource-tracing-1.0.jar" +HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/xtrace-2.1-20120824.jar" +HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/log4j-1.2.17.jar" +HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/aspectjrt-1.7.3.jar" From c10d329ccea6524decca0db313a12552aaeec63b Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 12 Nov 2013 12:15:08 -0800 Subject: [PATCH 037/112] Added the necessary command line arguments to put xresourcetracing on the bootclasspath, but left them commented out for now --- .../hadoop-common/src/main/bin/hadoop-config.cmd | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd index 3e6e4571a4eb..61689885df8c 100644 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd @@ -289,4 +289,10 @@ if not "%HADOOP_MAPRED_HOME%\%MAPRED_DIR%" == "%HADOOP_YARN_HOME%\%YARN_DIR%" ( set CLASSPATH=!CLASSPATH!;%HADOOP_MAPRED_HOME%\%MAPRED_DIR%\* ) +@rem XResourceTracing - add the XBootclasspath jars +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-resource-tracing-1.0.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-2.1-20120824.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\log4j-1.2.17.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\aspectjrt-1.7.3.jar + :eof From 739d0aada13b5fdc4293cf8cb4b178a714e5edc7 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 13 Nov 2013 13:54:39 -0800 Subject: [PATCH 038/112] We can now put xresourcetracing on the bootclasspath. It only takes effect if xresourcetracing was built to weave rt.jar --- .../hadoop-common/src/main/bin/hadoop-config.cmd | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd index 61689885df8c..ddc41835ee5f 100644 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd @@ -290,9 +290,9 @@ if not "%HADOOP_MAPRED_HOME%\%MAPRED_DIR%" == "%HADOOP_YARN_HOME%\%YARN_DIR%" ( ) @rem XResourceTracing - add the XBootclasspath jars -@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-resource-tracing-1.0.jar -@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-2.1-20120824.jar -@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\log4j-1.2.17.jar -@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\aspectjrt-1.7.3.jar +set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-resource-tracing-1.0.jar +set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-2.1-20120824.jar +set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\log4j-1.2.17.jar +set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\aspectjrt-1.7.3.jar :eof From 7b2fdedc706977a9de076ea704d398d7948c1f42 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 19 Nov 2013 13:42:59 -0500 Subject: [PATCH 039/112] Use a more accurate estimate for the PacketHeader size, now taking into account the fact that XTrace metadata can propagate options which might increase the PacketHeader size --- .../protocol/datatransfer/PacketHeader.java | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java index 9437862620f0..9d97b2cbb835 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java @@ -33,6 +33,7 @@ import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.ByteString; +import edu.berkeley.xtrace.OptionField; import edu.berkeley.xtrace.TaskID; import edu.berkeley.xtrace.XTraceContext; import edu.berkeley.xtrace.XTraceMetadata; @@ -56,15 +57,19 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class PacketHeader { - private static final int MAX_PROTO_SIZE = - PacketHeaderProto.newBuilder() - .setOffsetInBlock(0) - .setSeqno(0) - .setLastPacketInBlock(false) - .setDataLen(0) - .setSyncBlock(false) - .setXtrace(ByteString.copyFrom(new XTraceMetadata(new TaskID(8), 0L).pack())) - .build().getSerializedSize(); + private static final int MAX_PROTO_SIZE; + static { + XTraceMetadata max = new XTraceMetadata(new TaskID(8), 0L); + max.addOption(new OptionField((byte)0, new byte[254])); + MAX_PROTO_SIZE = PacketHeaderProto.newBuilder() + .setOffsetInBlock(0) + .setSeqno(0) + .setLastPacketInBlock(false) + .setDataLen(0) + .setSyncBlock(false) + .setXtrace(ByteString.copyFrom(max.pack())) + .build().getSerializedSize(); + } public static final int PKT_LENGTHS_LEN = Ints.BYTES + Shorts.BYTES; public static final int PKT_MAX_HEADER_LEN = From 242285c1a20c4f68d0d43476e1f7fcff085575e8 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 21 Nov 2013 20:45:27 -0500 Subject: [PATCH 040/112] Removed some hard-coded xtrace context passing that is now handled generically by the AspectJ instrumentation --- .../apache/hadoop/hdfs/DFSOutputStream.java | 26 +------------------ .../hdfs/server/datanode/BlockReceiver.java | 12 --------- 2 files changed, 1 insertion(+), 37 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 753b7abdd6ab..f04523df879c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -369,9 +369,6 @@ public DatanodeInfo load(DatanodeInfo key) throws Exception { /** Append on an existing block? */ private final boolean isAppend; - /** XTrace Context when streamer is created */ - private Collection xtrace = null; - /** * Default construction for file create */ @@ -471,8 +468,6 @@ private void endBlock() { */ @Override public void run() { - try { //xtrace try - long lastPacket = Time.now(); while (!streamerClosed && dfsClient.clientRunning) { @@ -481,7 +476,6 @@ public void run() { try { response.close(); response.join(); - response.joinXTraceContext(); response = null; } catch (InterruptedException e) { } @@ -654,10 +648,6 @@ public void run() { } } closeInternal(); - - } finally { // xtrace finally - xtrace = XTraceContext.getThreadContext(); - } } private void closeInternal() { @@ -670,10 +660,6 @@ private void closeInternal() { } } - public void joinXTraceContext() { - XTraceContext.joinContext(xtrace); - } - /* * close both streamer and DFSOutputStream, should be called only * by an external thread and only after all data to be sent has @@ -700,7 +686,6 @@ private void closeResponder() { response.join(); } catch (InterruptedException e) { } finally { - response.joinXTraceContext(); response = null; } } @@ -752,8 +737,6 @@ private class ResponseProcessor extends Daemon { @Override public void run() { - try { // xtrace try - setName("ResponseProcessor for block " + block); PipelineAck ack = new PipelineAck(); @@ -826,10 +809,6 @@ public void run() { } } } - - } finally { // xtrace finally - xtrace = XTraceContext.getThreadContext(); - } } void close() { @@ -837,9 +816,6 @@ void close() { this.interrupt(); } - void joinXTraceContext() { - XTraceContext.joinContext(xtrace); - } } // If this stream has encountered any errors so far, shutdown @@ -1873,6 +1849,7 @@ private void waitForAckedSeqno(long seqno) throws IOException { checkClosed(); if (lastAckedSeqno >= seqno) { XTraceContext.joinContext(lastAckedXTraceContext); + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Ack received, continuing", "lastAckedSeqno", lastAckedSeqno); break; } try { @@ -1910,7 +1887,6 @@ private void closeThreads(boolean force) throws IOException { try { streamer.close(force); streamer.join(); - streamer.joinXTraceContext(); if (s != null) { s.close(); } 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 6f9425adc512..f9ba572e8ba3 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 @@ -722,7 +722,6 @@ void receiveBlock( if (responder != null) { try { responder.join(datanode.getDnConf().getXceiverStopTimeout()); - ((PacketResponder)responder.getRunnable()).joinXtraceContext(); if (responder.isAlive()) { String msg = "Join on responder thread " + responder + " timed out"; @@ -855,8 +854,6 @@ class PacketResponder implements Runnable, Closeable { /** for log and error messages */ private final String myString; - private Collection xtrace = null; - @Override public String toString() { return myString; @@ -921,18 +918,12 @@ public synchronized void close() { notifyAll(); } - public void joinXtraceContext() { - XTraceContext.joinContext(xtrace); // rejoin the end xtrace context - } - /** * Thread to process incoming acks. * @see java.lang.Runnable#run() */ @Override public void run() { - try { // xtrace try - boolean lastPacketInBlock = false; final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0; while (running && datanode.shouldRun && !lastPacketInBlock) { @@ -1127,9 +1118,6 @@ public void run() { } } } - } finally { // xtrace finally - xtrace = XTraceContext.getThreadContext(); - } LOG.info(myString + " terminating"); } From 0145e24340ff4600522dd1a223ae5ac6eee0ae3a Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 11 Dec 2013 10:08:09 -0800 Subject: [PATCH 041/112] Clear the thread context in lease renewer, which is a long-lived thread and shouldn't be attributed to the first task that kicks it off --- .../src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java index 8f3b8fb0e325..b175c4ffc866 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java @@ -35,8 +35,11 @@ import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; + import com.google.common.annotations.VisibleForTesting; +import edu.berkeley.xtrace.XTraceContext; + /** *

    * Used by {@link DFSClient} for renewing file-being-written leases @@ -290,6 +293,10 @@ synchronized void put(final String src, final DFSOutputStream out, daemon = new Daemon(new Runnable() { @Override public void run() { + XTraceContext.logEvent(LeaseRenewer.class, "LeaseRenewer", "Lease renewer daemon for " + clientsString() + + " with renew id " + id + " started"); + XTraceContext.clearThreadContext(); // don't let the task id leak to lease renewer. + // if lease renewer tracing desired, should start a new task here and maybe log an event. try { if (LOG.isDebugEnabled()) { LOG.debug("Lease renewer daemon for " + clientsString() From 1c0d4ab1cbe6813f7add62e742bc8acf2e3e3866 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 16 Jan 2014 11:00:18 -0800 Subject: [PATCH 042/112] Fix to add causality when the sending thread has to wait to receive ACKs --- .../main/java/org/apache/hadoop/hdfs/DFSOutputStream.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index f04523df879c..44cd06ec27c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -1506,7 +1506,12 @@ private void queueCurrentPacket() { private void waitAndQueueCurrentPacket() throws IOException { synchronized (dataQueue) { // If queue is full, then wait till we have enough space + boolean first = true; while (!closed && dataQueue.size() + ackQueue.size() > MAX_PACKETS) { + if (first) { + XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Waiting for packets to be acked", "lastAckedSeqno", lastAckedSeqno); + first = false; + } try { dataQueue.wait(); } catch (InterruptedException e) { @@ -1520,6 +1525,8 @@ private void waitAndQueueCurrentPacket() throws IOException { Thread.currentThread().interrupt(); break; } + if (dataQueue.size() + ackQueue.size() > MAX_PACKETS) + XTraceContext.joinContext(lastAckedXTraceContext); } checkClosed(); queueCurrentPacket(); From c907d341519ec6166b6a5fdb28558637ff228c0a Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 16 Jan 2014 11:13:41 -0800 Subject: [PATCH 043/112] Fixed a bug - using less than instead of greater than --- .../src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 44cd06ec27c8..850ce4a1d635 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -1525,7 +1525,7 @@ private void waitAndQueueCurrentPacket() throws IOException { Thread.currentThread().interrupt(); break; } - if (dataQueue.size() + ackQueue.size() > MAX_PACKETS) + if (dataQueue.size() + ackQueue.size() <= MAX_PACKETS) XTraceContext.joinContext(lastAckedXTraceContext); } checkClosed(); From 0c858b71137beeb943c07f42c795c2a5ec90343b Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 27 Jan 2014 12:49:24 -0500 Subject: [PATCH 044/112] Removed and modified some of the XTraceContext.startTrace events, because they were polluting trace tags and annoying the hell out of me --- .../java/org/apache/hadoop/fs/FsShell.java | 2 +- .../org/apache/hadoop/hdfs/DFSClient.java | 34 +++++++++---------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java index cf4c861b2571..10d4029baa41 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java @@ -251,7 +251,7 @@ public int run(String argv[]) throws Exception { } else { String cmd = argv[0]; Command instance = null; - XTraceContext.startTrace("FsShell", "Executing Command", Joiner.on(" ").join(argv)); + XTraceContext.startTrace("FsShell", "Executing Command", "command line " + Joiner.on(" ").join(argv)); try { instance = commandFactory.getInstance(cmd); if (instance == null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 55b01252da07..77bdf8520f23 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1020,7 +1020,7 @@ public LocatedBlocks getLocatedBlocks(String src, long start, long length) static LocatedBlocks callGetBlockLocations(ClientProtocol namenode, String src, long start, long length) throws IOException { - XTraceContext.startTrace("DFSClient", "GetBlockLocations", "getBlockLocations", src+" ["+start+":"+(start+length)+"]"); + //XTraceContext.startTrace("DFSClient", "GetBlockLocations", "getBlockLocations", src+" ["+start+":"+(start+length)+"]"); try { return namenode.getBlockLocations(src, start, length); } catch(RemoteException re) { @@ -1171,7 +1171,7 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum, public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) throws IOException, UnresolvedLinkException { checkOpen(); - XTraceContext.startTrace("DFSClient", "Open", "open", src); + //XTraceContext.startTrace("DFSClient", "Open", "open", src); // Get block info from namenode return new DFSInputStream(this, src, buffersize, verifyChecksum); } @@ -1343,7 +1343,7 @@ public DFSOutputStream create(String src, + favoredNodes[i].getPort(); } } - XTraceContext.startTrace("DFSClient", "Create", "create", src); + //XTraceContext.startTrace("DFSClient", "Create", "create", src); final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this, src, masked, flag, createParent, replication, blockSize, progress, buffersize, dfsClientConf.createChecksum(checksumOpt), favoredNodeStrs); @@ -1366,7 +1366,7 @@ private DFSOutputStream primitiveAppend(String src, EnumSet flag, } return null; } - XTraceContext.startTrace("DFSClient", "Append", "append", src); + //XTraceContext.startTrace("DFSClient", "Append", "append", src); return callAppend(stat, src, buffersize, progress); } return null; @@ -1391,7 +1391,7 @@ public DFSOutputStream primitiveCreate(String src, CreateFlag.validate(flag); DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress); if (result == null) { - XTraceContext.startTrace("DFSClient", "Create", "create", src); + //XTraceContext.startTrace("DFSClient", "Create", "create", src); DataChecksum checksum = dfsClientConf.createChecksum(checksumOpt); result = DFSOutputStream.newStreamForCreate(this, src, absPermission, flag, createParent, replication, blockSize, progress, buffersize, @@ -1472,7 +1472,7 @@ private DFSOutputStream callAppend(HdfsFileStatus stat, String src, public HdfsDataOutputStream append(final String src, final int buffersize, final Progressable progress, final FileSystem.Statistics statistics ) throws IOException { - XTraceContext.startTrace("DFSClient", "Append", "append", src); + //XTraceContext.startTrace("DFSClient", "Append", "append", src); final DFSOutputStream out = append(src, buffersize, progress); return new HdfsDataOutputStream(out, statistics, out.getInitialLen()); } @@ -1520,7 +1520,7 @@ public boolean setReplication(String src, short replication) public boolean rename(String src, String dst) throws IOException { checkOpen(); try { - XTraceContext.startTrace("DFSClient", "Rename", "rename", src+" -> "+dst); + //XTraceContext.startTrace("DFSClient", "Rename", "rename", src+" -> "+dst); return namenode.rename(src, dst); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1540,7 +1540,7 @@ public boolean rename(String src, String dst) throws IOException { public void concat(String trg, String [] srcs) throws IOException { checkOpen(); try { - XTraceContext.startTrace("DFSClient", "Concat", "concat"); + //XTraceContext.startTrace("DFSClient", "Concat", "concat"); namenode.concat(trg, srcs); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1558,7 +1558,7 @@ public void rename(String src, String dst, Options.Rename... options) throws IOException { checkOpen(); try { - XTraceContext.startTrace("DFSClient", "Rename", "rename", src+" -> "+dst); + //XTraceContext.startTrace("DFSClient", "Rename", "rename", src+" -> "+dst); namenode.rename2(src, dst, options); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1581,7 +1581,7 @@ public void rename(String src, String dst, Options.Rename... options) @Deprecated public boolean delete(String src) throws IOException { checkOpen(); - XTraceContext.startTrace("DFSClient", "Delete", "delete", src); + //XTraceContext.startTrace("DFSClient", "Delete", "delete", src); try { // xtrace try return namenode.delete(src, true); } finally { @@ -1599,7 +1599,7 @@ public boolean delete(String src) throws IOException { public boolean delete(String src, boolean recursive) throws IOException { checkOpen(); try { - XTraceContext.startTrace("DFSClient", "Delete", "delete", src); + //XTraceContext.startTrace("DFSClient", "Delete", "delete", src); return namenode.delete(src, recursive); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1642,7 +1642,7 @@ public DirectoryListing listPaths(String src, byte[] startAfter, throws IOException { checkOpen(); try { - XTraceContext.startTrace("DFSClient", "ListPaths", "listPaths", src); + //XTraceContext.startTrace("DFSClient", "ListPaths", "listPaths", src); return namenode.getListing(src, startAfter, needLocation); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1664,7 +1664,7 @@ public DirectoryListing listPaths(String src, byte[] startAfter, public HdfsFileStatus getFileInfo(String src) throws IOException { checkOpen(); try { - XTraceContext.startTrace("DFSClient", "GetFileInfo", "getFileInfo", src); + //XTraceContext.startTrace("DFSClient", "GetFileInfo", "getFileInfo", src); return namenode.getFileInfo(src); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -2150,7 +2150,7 @@ public String createSnapshot(String snapshotRoot, String snapshotName) throws IOException { checkOpen(); try { - XTraceContext.startTrace("DFSClient", "CreateSnapshot", "createSnapshot", snapshotRoot); + //XTraceContext.startTrace("DFSClient", "CreateSnapshot", "createSnapshot", snapshotRoot); return namenode.createSnapshot(snapshotRoot, snapshotName); } catch(RemoteException re) { throw re.unwrapRemoteException(); @@ -2171,7 +2171,7 @@ public String createSnapshot(String snapshotRoot, String snapshotName) public void deleteSnapshot(String snapshotRoot, String snapshotName) throws IOException { try { - XTraceContext.startTrace("DFSClient", "DeleteSnapshot", "deleteSnapshot", snapshotRoot); + //XTraceContext.startTrace("DFSClient", "DeleteSnapshot", "deleteSnapshot", snapshotRoot); namenode.deleteSnapshot(snapshotRoot, snapshotName); } catch(RemoteException re) { throw re.unwrapRemoteException(); @@ -2192,7 +2192,7 @@ public void renameSnapshot(String snapshotDir, String snapshotOldName, String snapshotNewName) throws IOException { checkOpen(); try { - XTraceContext.startTrace("DFSClient", "RenameSnapshot", "renameSnapshot", snapshotOldName+" -> "+snapshotNewName); + //XTraceContext.startTrace("DFSClient", "RenameSnapshot", "renameSnapshot", snapshotOldName+" -> "+snapshotNewName); namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName); } catch(RemoteException re) { throw re.unwrapRemoteException(); @@ -2383,7 +2383,7 @@ public boolean primitiveMkdir(String src, FsPermission absPermission, boolean createParent) throws IOException { checkOpen(); - XTraceContext.startTrace("DFSClient", "MkDir", "mkdir", src); + //XTraceContext.startTrace("DFSClient", "MkDir", "mkdir", src); if (absPermission == null) { absPermission = FsPermission.getDefault().applyUMask(dfsClientConf.uMask); From 55bbdc6534315b663485c4414649af79c7ec0697 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 30 Jan 2014 13:01:49 -0500 Subject: [PATCH 045/112] Commented out the inclusion of resource tracing on the bootclasspath; it limits the classes that can be turned on and off in xtrace config. For now, we aren't using rt.jar instrumentation, and therefore no reason for these to be on the bootclasspath --- .../hadoop-common/src/main/bin/hadoop-config.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh index e409ac82f609..ce5768864a8c 100644 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh @@ -293,7 +293,7 @@ if [ "$HADOOP_CLASSPATH" != "" ]; then fi fi -HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/xtrace-resource-tracing-1.0.jar" -HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/xtrace-2.1-20120824.jar" -HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/log4j-1.2.17.jar" -HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/aspectjrt-1.7.3.jar" +#HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/xtrace-resource-tracing-1.0.jar" +#HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/xtrace-2.1-20120824.jar" +#HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/log4j-1.2.17.jar" +#HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/aspectjrt-1.7.3.jar" From 580097cc6c4db7c79e76cb383a7ca110e0d927df Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 14 Feb 2014 12:22:18 -0800 Subject: [PATCH 046/112] Commented out bootclasspath stuff, since for now we don't want or need it --- .../hadoop-common/src/main/bin/hadoop-config.cmd | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd index ddc41835ee5f..61689885df8c 100644 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd @@ -290,9 +290,9 @@ if not "%HADOOP_MAPRED_HOME%\%MAPRED_DIR%" == "%HADOOP_YARN_HOME%\%YARN_DIR%" ( ) @rem XResourceTracing - add the XBootclasspath jars -set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-resource-tracing-1.0.jar -set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-2.1-20120824.jar -set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\log4j-1.2.17.jar -set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\aspectjrt-1.7.3.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-resource-tracing-1.0.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-2.1-20120824.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\log4j-1.2.17.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\aspectjrt-1.7.3.jar :eof From 652191513b6c006b7bd2713a0cd8238eaaf21d98 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 18 Feb 2014 10:39:27 -0800 Subject: [PATCH 047/112] More network instrumentation for RPC calls --- .../src/main/java/org/apache/hadoop/ipc/Server.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index 31656658c68f..11cc41c499d1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -116,6 +116,8 @@ import edu.berkeley.xtrace.XTraceContext; import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.resourcetracing.loggers.xtrace.NetworkIO; +import edu.brown.cs.systems.xtrace.resourcetracing.loggers.pubsub.NetworkIOAggregation; /** An abstract IPC service. IPC calls take a single {@link Writable} as a * parameter, and return a {@link Writable} as their value. A service runs on @@ -482,6 +484,7 @@ public static class Call { private final RPC.RpcKind rpcKind; private final byte[] clientId; private Collection xtrace; // the X-Trace context this was received with + private Collection responsextrace; // the X-Trace context before sending the response public Call(int id, int retryCount, Writable param, Connection connection) { @@ -1010,6 +1013,7 @@ private boolean processResponse(LinkedList responseQueue, // Extract the first call // call = responseQueue.removeFirst(); + XTraceContext.joinContext(call.responsextrace); SocketChannel channel = call.connection.channel; if (LOG.isDebugEnabled()) { LOG.debug(getName() + ": responding to " + call); @@ -1790,6 +1794,8 @@ private void processOneRpc(byte[] buf) LOG.debug(" got #" + callId); } checkRpcHeaders(header); + NetworkIO.didReadBefore(this, buf.length); + NetworkIOAggregation.didReadBefore(buf.length); if (callId < 0) { // callIds typically used during connection setup processRpcOutOfBandRequest(header, dis); @@ -2375,7 +2381,8 @@ private void setupResponse(ByteArrayOutputStream responseBuf, call.setResponse(ByteBuffer.wrap(responseBuf.toByteArray())); if (XTraceContext.isValid()) { - XTraceContext.clearThreadContext(); //to prevent leaking + call.responsextrace = XTraceContext.getThreadContext(); +// XTraceContext.clearThreadContext(); //to prevent leaking } } From e642c28d83bd93e6a50d4f617a34434166db5dcb Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 18 Feb 2014 10:39:46 -0800 Subject: [PATCH 048/112] Added a few entries to the default hdfs config; by default, there are a few things we want turned off --- .../hadoop-hdfs/src/main/conf/hdfs-site.xml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml index 50ec1460bd60..776c540e7cde 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml @@ -17,5 +17,21 @@ + + dfs.namenode.fs-limits.min-block-size + 65536 + + + dfs.datanode.readahead.bytes + 0 + + + dfs.datanode.drop.cache.behind.writes + true + + + dfs.datanode.sync.behind.writes + true + From 3320391d5ab30a027ffc0f428cb5af415dd52d3f Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 18 Feb 2014 12:57:31 -0800 Subject: [PATCH 049/112] Testing the write speed with a lower cache drop behind buffer lag --- .../org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 f9ba572e8ba3..c11edfbdc0bd 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 @@ -65,7 +65,8 @@ class BlockReceiver implements Closeable { 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 From 4db14a2cd58bfc3fa0708b661b08a1d18ba92240 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 28 Feb 2014 12:19:39 -0800 Subject: [PATCH 050/112] Applied broken pom patch from HADOOP-10110; hasn't affected us thus far but sandbox build was failing --- hadoop-common-project/hadoop-auth/pom.xml | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/hadoop-common-project/hadoop-auth/pom.xml b/hadoop-common-project/hadoop-auth/pom.xml index 5f474dd02125..75f0d64b55fb 100644 --- a/hadoop-common-project/hadoop-auth/pom.xml +++ b/hadoop-common-project/hadoop-auth/pom.xml @@ -52,12 +52,17 @@ org.mockito mockito-all test - + - org.mortbay.jetty - jetty - test - + org.mortbay.jetty + jetty-util + test + + + org.mortbay.jetty + jetty + test + javax.servlet servlet-api From a7019c7b71286083516bebf7809aa9a6792d4eb6 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 11 Mar 2014 07:19:52 -0700 Subject: [PATCH 051/112] Temporary instrumentation adding additional events to DataXceiver --- .../apache/hadoop/hdfs/server/datanode/DataXceiver.java | 7 +++++++ 1 file changed, 7 insertions(+) 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 e2f720cd795a..eb4934a8defc 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 @@ -330,6 +330,7 @@ public void readBlock(final ExtendedBlock block, remoteAddress; updateCurrentThreadName("Sending block " + block); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Creating BlockSender"); try { try { blockSender = new BlockSender(block, blockOffset, length, @@ -341,11 +342,15 @@ public void readBlock(final ExtendedBlock block, throw e; } + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "WritingRPCResult"); + // send op status writeSuccessWithChecksumInfo(blockSender, new DataOutputStream(getOutputStream())); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sending Block"); long read = blockSender.sendBlock(out, baseStream, null); // send data + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sent Block"); if (blockSender.didSendEntireByteRange()) { // If we sent the entire range, then we should expect the client // to respond with a Status enum. @@ -363,8 +368,10 @@ public void readBlock(final ExtendedBlock block, LOG.debug("Error reading client status response. Will close connection.", ioe); IOUtils.closeStream(out); } + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Received RPC status enum"); } else { IOUtils.closeStream(out); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Closed Stream"); } datanode.metrics.incrBytesRead((int) read); datanode.metrics.incrBlocksRead(); From 8af128da59674540b45fcec96157e9d5ba866fb3 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 11 Mar 2014 08:55:28 -0700 Subject: [PATCH 052/112] Undo previous commit --- .../apache/hadoop/hdfs/server/datanode/DataXceiver.java | 7 ------- 1 file changed, 7 deletions(-) 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 eb4934a8defc..e2f720cd795a 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 @@ -330,7 +330,6 @@ public void readBlock(final ExtendedBlock block, remoteAddress; updateCurrentThreadName("Sending block " + block); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Creating BlockSender"); try { try { blockSender = new BlockSender(block, blockOffset, length, @@ -342,15 +341,11 @@ public void readBlock(final ExtendedBlock block, throw e; } - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "WritingRPCResult"); - // send op status writeSuccessWithChecksumInfo(blockSender, new DataOutputStream(getOutputStream())); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sending Block"); long read = blockSender.sendBlock(out, baseStream, null); // send data - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sent Block"); if (blockSender.didSendEntireByteRange()) { // If we sent the entire range, then we should expect the client // to respond with a Status enum. @@ -368,10 +363,8 @@ public void readBlock(final ExtendedBlock block, LOG.debug("Error reading client status response. Will close connection.", ioe); IOUtils.closeStream(out); } - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Received RPC status enum"); } else { IOUtils.closeStream(out); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Closed Stream"); } datanode.metrics.incrBytesRead((int) read); datanode.metrics.incrBlocksRead(); From e9e159864084389846ba8cae80de2cdd1804fa28 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 11 Mar 2014 09:20:52 -0700 Subject: [PATCH 053/112] Add some more temporary logging --- .../apache/hadoop/hdfs/server/datanode/BlockSender.java | 2 ++ .../apache/hadoop/hdfs/server/datanode/DataXceiver.java | 7 +++++++ 2 files changed, 9 insertions(+) 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 f3749e617307..d59333b3394e 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 @@ -228,7 +228,9 @@ class BlockSender implements java.io.Closeable { */ DataChecksum csum = null; if (verifyChecksum || sendChecksum) { + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Getting metadata input stream"); final InputStream metaIn = datanode.data.getMetaDataInputStream(block); + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Got metadata input stream"); if (!corruptChecksumOk || metaIn != null) { if (metaIn == null) { //need checksum but meta-data not found 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 e2f720cd795a..eb4934a8defc 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 @@ -330,6 +330,7 @@ public void readBlock(final ExtendedBlock block, remoteAddress; updateCurrentThreadName("Sending block " + block); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Creating BlockSender"); try { try { blockSender = new BlockSender(block, blockOffset, length, @@ -341,11 +342,15 @@ public void readBlock(final ExtendedBlock block, throw e; } + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "WritingRPCResult"); + // send op status writeSuccessWithChecksumInfo(blockSender, new DataOutputStream(getOutputStream())); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sending Block"); long read = blockSender.sendBlock(out, baseStream, null); // send data + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sent Block"); if (blockSender.didSendEntireByteRange()) { // If we sent the entire range, then we should expect the client // to respond with a Status enum. @@ -363,8 +368,10 @@ public void readBlock(final ExtendedBlock block, LOG.debug("Error reading client status response. Will close connection.", ioe); IOUtils.closeStream(out); } + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Received RPC status enum"); } else { IOUtils.closeStream(out); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Closed Stream"); } datanode.metrics.incrBytesRead((int) read); datanode.metrics.incrBlocksRead(); From d8d8727210e37d44bd13a3996b305ba4819b5e07 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 11 Mar 2014 10:53:06 -0700 Subject: [PATCH 054/112] Removed some logging that was temporary --- .../apache/hadoop/hdfs/server/datanode/BlockSender.java | 2 -- .../apache/hadoop/hdfs/server/datanode/DataXceiver.java | 7 ------- 2 files changed, 9 deletions(-) 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 d59333b3394e..f3749e617307 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 @@ -228,9 +228,7 @@ class BlockSender implements java.io.Closeable { */ DataChecksum csum = null; if (verifyChecksum || sendChecksum) { - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Getting metadata input stream"); final InputStream metaIn = datanode.data.getMetaDataInputStream(block); - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Got metadata input stream"); if (!corruptChecksumOk || metaIn != null) { if (metaIn == null) { //need checksum but meta-data not found 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 eb4934a8defc..e2f720cd795a 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 @@ -330,7 +330,6 @@ public void readBlock(final ExtendedBlock block, remoteAddress; updateCurrentThreadName("Sending block " + block); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Creating BlockSender"); try { try { blockSender = new BlockSender(block, blockOffset, length, @@ -342,15 +341,11 @@ public void readBlock(final ExtendedBlock block, throw e; } - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "WritingRPCResult"); - // send op status writeSuccessWithChecksumInfo(blockSender, new DataOutputStream(getOutputStream())); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sending Block"); long read = blockSender.sendBlock(out, baseStream, null); // send data - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sent Block"); if (blockSender.didSendEntireByteRange()) { // If we sent the entire range, then we should expect the client // to respond with a Status enum. @@ -368,10 +363,8 @@ public void readBlock(final ExtendedBlock block, LOG.debug("Error reading client status response. Will close connection.", ioe); IOUtils.closeStream(out); } - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Received RPC status enum"); } else { IOUtils.closeStream(out); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Closed Stream"); } datanode.metrics.incrBytesRead((int) read); datanode.metrics.incrBlocksRead(); From a5f52b84b13d4d9e689461829ce85ed384b1b508 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 11 Mar 2014 11:33:20 -0700 Subject: [PATCH 055/112] Revert "Removed some logging that was temporary" This reverts commit d8d8727210e37d44bd13a3996b305ba4819b5e07. --- .../apache/hadoop/hdfs/server/datanode/BlockSender.java | 2 ++ .../apache/hadoop/hdfs/server/datanode/DataXceiver.java | 7 +++++++ 2 files changed, 9 insertions(+) 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 f3749e617307..d59333b3394e 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 @@ -228,7 +228,9 @@ class BlockSender implements java.io.Closeable { */ DataChecksum csum = null; if (verifyChecksum || sendChecksum) { + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Getting metadata input stream"); final InputStream metaIn = datanode.data.getMetaDataInputStream(block); + XTraceContext.logEvent(BlockSender.class, "BlockSender", "Got metadata input stream"); if (!corruptChecksumOk || metaIn != null) { if (metaIn == null) { //need checksum but meta-data not found 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 e2f720cd795a..eb4934a8defc 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 @@ -330,6 +330,7 @@ public void readBlock(final ExtendedBlock block, remoteAddress; updateCurrentThreadName("Sending block " + block); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Creating BlockSender"); try { try { blockSender = new BlockSender(block, blockOffset, length, @@ -341,11 +342,15 @@ public void readBlock(final ExtendedBlock block, throw e; } + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "WritingRPCResult"); + // send op status writeSuccessWithChecksumInfo(blockSender, new DataOutputStream(getOutputStream())); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sending Block"); long read = blockSender.sendBlock(out, baseStream, null); // send data + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sent Block"); if (blockSender.didSendEntireByteRange()) { // If we sent the entire range, then we should expect the client // to respond with a Status enum. @@ -363,8 +368,10 @@ public void readBlock(final ExtendedBlock block, LOG.debug("Error reading client status response. Will close connection.", ioe); IOUtils.closeStream(out); } + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Received RPC status enum"); } else { IOUtils.closeStream(out); + XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Closed Stream"); } datanode.metrics.incrBytesRead((int) read); datanode.metrics.incrBlocksRead(); From a87d2681893a7a611857b40585f7447eb306b1b0 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 24 Mar 2014 16:28:13 -0700 Subject: [PATCH 056/112] Migration from X-Trace 2.0 to X-Trace 3.0. Preliminary commit of untested instrumentation --- .../org/apache/hadoop/fs/FSOutputSummer.java | 8 -- .../java/org/apache/hadoop/fs/FsShell.java | 13 +- .../java/org/apache/hadoop/ipc/Client.java | 63 ++++----- .../apache/hadoop/ipc/ProtobufRpcEngine.java | 31 ++--- .../java/org/apache/hadoop/ipc/Server.java | 66 ++++------ .../apache/hadoop/ipc/WritableRpcEngine.java | 57 +++++---- .../java/org/apache/hadoop/net/NetUtils.java | 29 +++-- .../org/apache/hadoop/util/ProtoUtil.java | 11 +- .../java/org/apache/hadoop/util/Shell.java | 23 ++-- .../org/apache/hadoop/hdfs/DFSClient.java | 46 +------ .../apache/hadoop/hdfs/DFSOutputStream.java | 71 +++++----- .../org/apache/hadoop/hdfs/LeaseRenewer.java | 8 +- .../apache/hadoop/hdfs/RemoteBlockReader.java | 3 - .../hadoop/hdfs/RemoteBlockReader2.java | 11 +- .../hdfs/protocol/XTraceProtoUtils.java | 30 ++--- .../datatransfer/DataTransferProtoUtil.java | 8 +- .../protocol/datatransfer/PacketHeader.java | 38 +++--- .../protocol/datatransfer/PacketReceiver.java | 21 +-- .../protocol/datatransfer/PipelineAck.java | 21 ++- .../hdfs/protocol/datatransfer/Receiver.java | 17 +-- .../hdfs/server/datanode/BPServiceActor.java | 5 +- .../hdfs/server/datanode/BlockReceiver.java | 40 +++--- .../hdfs/server/datanode/BlockSender.java | 23 ++-- .../hadoop/hdfs/server/datanode/DataNode.java | 3 + .../hdfs/server/datanode/DataXceiver.java | 56 ++++---- .../hadoop/mapred/LocalContainerLauncher.java | 12 +- .../mapred/TaskAttemptListenerImpl.java | 9 +- .../org/apache/hadoop/mapred/YarnChild.java | 23 ++-- .../jobhistory/JobHistoryEventHandler.java | 6 +- .../hadoop/mapreduce/v2/app/MRAppMaster.java | 31 ++--- .../v2/app/TaskHeartbeatHandler.java | 6 +- .../mapreduce/v2/app/job/impl/JobImpl.java | 101 ++++++++------- .../app/launcher/ContainerLauncherImpl.java | 14 +- .../mapreduce/v2/app/rm/RMCommunicator.java | 6 +- .../v2/app/rm/RMContainerAllocator.java | 89 ++++++------- .../v2/app/rm/RMContainerRequestor.java | 26 ++-- .../v2/app/speculate/DefaultSpeculator.java | 6 - .../apache/hadoop/mapred/LocalJobRunner.java | 29 ++--- .../v2/api/records/TaskAttemptId.java | 12 +- .../apache/hadoop/mapred/CleanupQueue.java | 8 -- .../org/apache/hadoop/mapred/IndexRecord.java | 38 +++--- .../org/apache/hadoop/mapred/MapTask.java | 121 ++++++++---------- .../java/org/apache/hadoop/mapred/Merger.java | 8 +- .../org/apache/hadoop/mapred/ReduceTask.java | 11 +- .../org/apache/hadoop/mapred/SpillRecord.java | 3 - .../java/org/apache/hadoop/mapred/Task.java | 97 +++++++------- .../mapred/lib/MultithreadedMapRunner.java | 27 ++-- .../org/apache/hadoop/mapreduce/Cluster.java | 1 - .../java/org/apache/hadoop/mapreduce/Job.java | 38 +++--- .../hadoop/mapreduce/JobSubmissionFiles.java | 10 +- .../apache/hadoop/mapreduce/JobSubmitter.java | 20 +-- .../hadoop/mapreduce/lib/chain/Chain.java | 10 -- .../lib/map/MultithreadedMapper.java | 21 +-- .../hadoop/mapreduce/task/reduce/Fetcher.java | 74 +++++------ .../task/reduce/InMemoryMapOutput.java | 17 +-- .../mapreduce/task/reduce/MapOutput.java | 16 +-- .../task/reduce/MergeManagerImpl.java | 84 ++++++------ .../mapreduce/task/reduce/MergeThread.java | 5 +- .../task/reduce/OnDiskMapOutput.java | 18 +-- .../hadoop/mapreduce/task/reduce/Shuffle.java | 28 ++-- .../mapreduce/task/reduce/ShuffleHeader.java | 31 +++-- .../task/reduce/ShuffleSchedulerImpl.java | 19 ++- .../mapreduce/v2/hs/HistoryFileManager.java | 6 - .../hadoop/mapreduce/v2/hs/JobHistory.java | 20 --- .../hadoop/mapred/ResourceMgrDelegate.java | 8 +- .../apache/hadoop/mapred/ShuffleHandler.java | 30 ++--- hadoop-project-dist/pom.xml | 4 +- .../yarn/YarnUncaughtExceptionHandler.java | 14 +- .../records/impl/pb/ContainerIdPBImpl.java | 29 ++--- .../records/impl/pb/LocalResourcePBImpl.java | 21 +-- .../impl/pb/ResourceRequestPBImpl.java | 21 +-- .../hadoop/yarn/event/AbstractEvent.java | 12 +- .../hadoop/yarn/event/AsyncDispatcher.java | 25 ++-- .../impl/pb/RpcClientFactoryPBImpl.java | 1 - .../yarn/state/StateMachineFactory.java | 25 ++-- .../apache/hadoop/yarn/util/FSDownload.java | 26 ++-- .../nodemanager/DefaultContainerExecutor.java | 17 ++- .../server/nodemanager/DeletionService.java | 10 -- .../nodemanager/LinuxContainerExecutor.java | 21 +-- .../nodemanager/NodeStatusUpdaterImpl.java | 6 +- .../impl/pb/LocalResourceStatusPBImpl.java | 24 ++-- .../ContainerManagerImpl.java | 9 +- .../application/ApplicationImpl.java | 16 +-- .../container/ContainerImpl.java | 14 +- .../launcher/ContainerLaunch.java | 10 +- .../launcher/ContainersLauncher.java | 7 +- .../localizer/ContainerLocalizer.java | 20 +-- .../ResourceLocalizationService.java | 47 +++---- .../loghandler/NonAggregatingLogHandler.java | 8 -- .../monitor/ContainersMonitorImpl.java | 15 +-- .../ApplicationMasterService.java | 5 +- .../resourcemanager/ClientRMService.java | 10 +- .../server/resourcemanager/RMAppManager.java | 5 +- .../resourcemanager/ResourceManager.java | 9 +- .../amlauncher/AMLauncher.java | 22 +--- .../resourcemanager/rmapp/RMAppEvent.java | 4 +- .../rmapp/attempt/RMAppAttemptImpl.java | 5 +- .../resourcemanager/rmnode/RMNodeImpl.java | 12 +- .../scheduler/capacity/CapacityScheduler.java | 13 +- .../scheduler/capacity/LeafQueue.java | 6 +- .../common/fica/FiCaSchedulerApp.java | 10 +- .../scheduler/fair/FairScheduler.java | 17 ++- .../scheduler/fifo/FifoScheduler.java | 32 +++-- 103 files changed, 1072 insertions(+), 1321 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java index 663649153c79..4b7e545c047c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java @@ -25,8 +25,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import edu.berkeley.xtrace.XTraceContext; - /** * This is a generic output stream for generating checksums for * data before it is written to the underlying stream @@ -97,8 +95,6 @@ public synchronized void write(int b) throws IOException { public synchronized void write(byte b[], int off, int len) throws IOException { - try { // xtrace try - checkClosed(); if (off < 0 || len < 0 || off > b.length - len) { @@ -108,10 +104,6 @@ public synchronized void write(byte b[], int off, int len) for (int n=0;n xtrace; // X-Trace context for the return + Context xtrace; // X-Trace context for the return private Call(RPC.RpcKind rpcKind, Writable param) { this.rpcKind = rpcKind; @@ -638,8 +637,8 @@ private synchronized void setupIOstreams() { if (socket != null || shouldCloseConnection.get()) { return; } - XTraceContext.logEvent(Connection.class, "RPC Client", "Connecting to server"); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("Connecting to server"); + Context start_context = XTrace.get(); try { if (LOG.isDebugEnabled()) { LOG.debug("Connecting to "+server); @@ -711,13 +710,13 @@ public AuthMethod run() // start the receiver thread after the socket connection has been set // up start(); - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(Connection.class, "RPC Client", "Connected to server"); + XTrace.join(start_context); + xtrace.log("Connected to server"); return; } } catch (Throwable t) { - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(Connection.class, "RPC Client", "Failed to connect to server: "+t.getClass().getName(), "Message", t.getMessage()); + XTrace.join(start_context); + xtrace.log("Failed to connect to server: "+t.getClass().getName(), "Message", t.getMessage()); if (t instanceof IOException) { markClosed((IOException)t); } else { @@ -895,11 +894,13 @@ private synchronized void sendPing() throws IOException { @Override public void run() { - XTraceContext.logEvent(this.getClass(), "Client Connection", "RPC Response reader thread started"); + // XTrace: this is a long-lived thread that is lazily started by the first client. + // Clear any thread contexts that may have been set after the thread begins + xtrace.log("RPC Response reader thread started"); if (LOG.isDebugEnabled()) LOG.debug(getName() + ": starting, having connections " + connections.size()); - XTraceContext.clearThreadContext(); + XTrace.stop(); try { while (waitForWork()) {//wait here for work - read or close connection receiveRpcResponse(); @@ -960,7 +961,7 @@ public void run() { if (LOG.isDebugEnabled()) LOG.debug(getName() + " sending #" + call.id); - XTraceContext.logEvent(Client.Connection.class, "Client senderFuture", "Sending call"); + xtrace.log("Client senderFuture sending call"); byte[] data = d.getData(); int totalLength = d.getLength(); @@ -1023,11 +1024,7 @@ private void receiveRpcResponse() { RpcStatusProto status = header.getStatus(); if (header.hasXtrace()) { ByteString xbs = header.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), - 0, xbs.size()); - if (xmd.isValid()) { - call.xtrace = new XTraceMetadataCollection(xmd); - } + call.xtrace = Context.parse(xbs.toByteArray()); } if (status == RpcStatusProto.SUCCESS) { @@ -1340,10 +1337,8 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, ConnectionId remoteId, int serviceClass) throws IOException { final Call call = createCall(rpcKind, rpcRequest); Connection connection = getConnection(remoteId, call, serviceClass); - if (XTraceContext.isValid()) { - XTraceContext.logEvent(RPC.class, "RPC Client", "Sending RPC request", "Call ID", call.id); - call.xtrace = XTraceContext.getThreadContext(); - } + xtrace.log("Sending RPC request", "Call ID", call.id); + call.xtrace = XTrace.get(); try { connection.sendRpcRequest(call); // send the rpc request } catch (RejectedExecutionException e) { @@ -1369,16 +1364,11 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, Thread.currentThread().interrupt(); } - if (call.xtrace != null) { - XTraceContext.joinContext(call.xtrace); - } + XTrace.join(call.xtrace); if (call.error != null) { if (call.error instanceof RemoteException) { call.error.fillInStackTrace(); - if (XTraceContext.isValid()) { - XTraceContext.logEvent(RPC.class, "RPC Client", "RPC response received remote exception", - "Call ID", call.id, "Message", call.error.getMessage()); - } + xtrace.log("RPC response received remote exception", "Call ID", call.id, "Message", call.error.getMessage()); throw call.error; } else { // local exception InetSocketAddress address = connection.getRemoteAddress(); @@ -1387,16 +1377,11 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, NetUtils.getHostname(), 0, call.error); - if (XTraceContext.isValid()) { - XTraceContext.logEvent(RPC.class, "RPC Client", "Local exception handling RPC response", - "Call ID", call.id, "Message", e.getMessage()); - } + xtrace.log("Local exception handling RPC response", "Call ID", call.id, "Message", e.getMessage()); throw e; } } else { - if (XTraceContext.isValid()) { - XTraceContext.logEvent(RPC.class, "RPC Client", "Received RPC response", "Call ID", call.id); - } + xtrace.log("Received RPC response", "Call ID", call.id); return call.getRpcResponse(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java index 96961ef243a8..0b66a05f373d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java @@ -25,7 +25,6 @@ import java.lang.reflect.Method; import java.lang.reflect.Proxy; import java.net.InetSocketAddress; -import java.util.Collection; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -59,14 +58,16 @@ import com.google.protobuf.ServiceException; import com.google.protobuf.TextFormat; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; + /** * RPC Engine for for protobuf based RPCs. */ @InterfaceStability.Evolving public class ProtobufRpcEngine implements RpcEngine { + public static final XTrace.Logger xtrace = XTrace.getLogger(ProtobufRpcEngine.class); public static final Log LOG = LogFactory.getLog(ProtobufRpcEngine.class); static { // Register the rpcRequest deserializer for WritableRpcEngine @@ -181,8 +182,8 @@ private RequestHeaderProto constructRpcRequestHeader(Method method) { public Object invoke(Object proxy, Method method, Object[] args) throws ServiceException { - XTraceContext.logEvent(RPC.class, "ProtobufRpcEngine", "RPC Client invoking remote method "+method.getName(), "Protocol", this.protocolName, "ConnectionID", this.remoteId); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("Invoking remote method "+method.getName(), "Protocol", this.protocolName, "ConnectionID", this.remoteId); + Context start_context = XTrace.get(); try { // xtrace try long startTime = 0; @@ -251,13 +252,13 @@ public Object invoke(Object proxy, Method method, Object[] args) throw new ServiceException(e); } - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(RPC.class, "ProtobufRpcEngine", "Client invocation of "+method.getName()+" successful"); + XTrace.join(start_context); + xtrace.log("Client invocation of "+method.getName()+" successful"); return returnMessage; } catch (ServiceException e) {// xtrace catch - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(RPC.class, "ProtobufRpcEngine", "Remote invocation of "+method.getName()+" failed due to exception: "+e.getClass().getName(), "Message", e.getMessage()); + XTrace.join(start_context); + xtrace.log("Remote invocation of "+method.getName()+" failed due to exception: "+e.getClass().getName(), "Message", e.getMessage()); throw e; } } @@ -581,8 +582,8 @@ public Writable call(RPC.Server server, String protocol, if (server.verbose) LOG.info("Call: protocol=" + protocol + ", method=" + methodName); - XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Invoking method "+methodName, "Protocol", protocol, "Name", "RPC: "+methodName); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("Invoking method "+methodName, "Protocol", protocol, "Name", "RPC: "+methodName); + Context start_context = XTrace.get(); try { // xtrace try ProtoClassProtoImpl protocolImpl = getProtocolImpl(server, protoName, @@ -621,12 +622,12 @@ public Writable call(RPC.Server server, String protocol, throw e; } - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Invocation of "+methodName+" completed, responding to client"); + XTrace.join(start_context); + xtrace.log("Invocation of "+methodName+" completed, responding to client"); return new RpcResponseWrapper(result); } catch (Exception e) { // xtrace catch - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(ProtoBufRpcInvoker.class, "ProtoBufRpcInvoker", "Failed to invoke method "+methodName+": "+e.getClass().getName(), "Message", e.getMessage()); + XTrace.join(start_context); + xtrace.log("Failed to invoke method "+methodName+": "+e.getClass().getName(), "Message", e.getMessage()); throw e; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index 11cc41c499d1..c03b15cbb834 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -18,6 +18,11 @@ package org.apache.hadoop.ipc; +import static org.apache.hadoop.ipc.RpcConstants.AUTHORIZATION_FAILED_CALL_ID; +import static org.apache.hadoop.ipc.RpcConstants.CONNECTION_CONTEXT_CALL_ID; +import static org.apache.hadoop.ipc.RpcConstants.CURRENT_VERSION; +import static org.apache.hadoop.ipc.RpcConstants.PING_CALL_ID; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -44,7 +49,6 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -72,8 +76,6 @@ import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; -import static org.apache.hadoop.ipc.RpcConstants.*; - import org.apache.hadoop.ipc.ProtobufRpcEngine.RpcResponseMessageWrapper; import org.apache.hadoop.ipc.ProtobufRpcEngine.RpcResponseWrapper; import org.apache.hadoop.ipc.RPC.RpcInvoker; @@ -114,10 +116,9 @@ import com.google.protobuf.Message; import com.google.protobuf.Message.Builder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.brown.cs.systems.xtrace.resourcetracing.loggers.xtrace.NetworkIO; -import edu.brown.cs.systems.xtrace.resourcetracing.loggers.pubsub.NetworkIOAggregation; +import edu.brown.cs.systems.resourcetracing.resources.Network; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** An abstract IPC service. IPC calls take a single {@link Writable} as a * parameter, and return a {@link Writable} as their value. A service runs on @@ -241,7 +242,7 @@ public static RpcInvoker getRpcInvoker(RPC.RpcKind rpcKind) { return (val == null) ? null : val.rpcInvoker; } - + public static final XTrace.Logger xtrace = XTrace.getLogger(Server.class); public static final Log LOG = LogFactory.getLog(Server.class); public static final Log AUDITLOG = LogFactory.getLog("SecurityLogger."+Server.class.getName()); @@ -483,8 +484,8 @@ public static class Call { private ByteBuffer rpcResponse; // the response for this call private final RPC.RpcKind rpcKind; private final byte[] clientId; - private Collection xtrace; // the X-Trace context this was received with - private Collection responsextrace; // the X-Trace context before sending the response + private Context start_context; // the X-Trace context this was received with + private Context response_context; // the X-Trace context before sending the response public Call(int id, int retryCount, Writable param, Connection connection) { @@ -502,10 +503,8 @@ public Call(int id, int retryCount, Writable param, Connection connection, this.rpcResponse = null; this.rpcKind = kind; this.clientId = clientId; - if (XTraceContext.isValid()) { - XTraceContext.logEvent(RPC.class, "RPC Server", "Received RPC call", "CallID", id); - this.xtrace = XTraceContext.getThreadContext(); - } + Server.xtrace.log("Received RPC call", "CallID", id); + this.start_context = XTrace.get(); } @Override @@ -1013,7 +1012,7 @@ private boolean processResponse(LinkedList responseQueue, // Extract the first call // call = responseQueue.removeFirst(); - XTraceContext.joinContext(call.responsextrace); + XTrace.join(call.response_context); SocketChannel channel = call.connection.channel; if (LOG.isDebugEnabled()) { LOG.debug(getName() + ": responding to " + call); @@ -1794,8 +1793,10 @@ private void processOneRpc(byte[] buf) LOG.debug(" got #" + callId); } checkRpcHeaders(header); - NetworkIO.didReadBefore(this, buf.length); - NetworkIOAggregation.didReadBefore(buf.length); + + // XTrace: one of the few places in hdfs source code where we have to explicitly call resource instrumentation code + Network.Read.alreadyStarted(this); + Network.Read.alreadyFinished(this, buf.length); if (callId < 0) { // callIds typically used during connection setup processRpcOutOfBandRequest(header, dis); @@ -1815,7 +1816,7 @@ private void processOneRpc(byte[] buf) responder.doRespond(call); throw wrse; } finally { - XTraceContext.clearThreadContext(); + XTrace.stop(); } } @@ -1826,14 +1827,9 @@ private void processOneRpc(byte[] buf) */ private void checkRpcHeaders(RpcRequestHeaderProto header) throws WrappedRpcServerException { - if (header.hasXtrace()) { - ByteString xbs = header.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), - 0, xbs.size()); - if (xmd.isValid()) { - XTraceContext.setThreadContext(xmd); - } - } + if (header.hasXtrace()) + XTrace.set(header.getXtrace().toByteArray()); + if (!header.hasRpcOp()) { String err = " IPC Server: No rpc op in rpcRequestHeader"; throw new WrappedRpcServerException( @@ -2037,12 +2033,10 @@ public void run() { ByteArrayOutputStream buf = new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE); while (running) { - XTraceContext.clearThreadContext(); + XTrace.stop(); try { final Call call = callQueue.take(); // pop the queue; maybe blocked here - if (call.xtrace != null) { - XTraceContext.setThreadContext(call.xtrace); - } + XTrace.set(call.start_context); if (LOG.isDebugEnabled()) { LOG.debug(getName() + ": " + call + " for RpcKind " + call.rpcKind); } @@ -2320,9 +2314,9 @@ private void setupResponse(ByteArrayOutputStream responseBuf, headerBuilder.setRetryCount(call.retryCount); headerBuilder.setStatus(status); headerBuilder.setServerIpcVersionNum(CURRENT_VERSION); - if (XTraceContext.isValid()) { - headerBuilder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); - } + if (XTrace.active()) + headerBuilder.setXtrace(ByteString.copyFrom(XTrace.bytes())); + /* X-Trace: we have to send in the response the last event in the server * before the data is sent, and this is not it, there can be more events * later, related to enqueuing and sending this call. To log them correctly @@ -2379,11 +2373,7 @@ private void setupResponse(ByteArrayOutputStream responseBuf, wrapWithSasl(responseBuf, call); } call.setResponse(ByteBuffer.wrap(responseBuf.toByteArray())); - - if (XTraceContext.isValid()) { - call.responsextrace = XTraceContext.getThreadContext(); -// XTraceContext.clearThreadContext(); //to prevent leaking - } + call.response_context = XTrace.get(); } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java index 441297dc60d9..bc1a566f8b1f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java @@ -18,36 +18,40 @@ package org.apache.hadoop.ipc; -import java.lang.reflect.Proxy; -import java.lang.reflect.Method; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.lang.reflect.InvocationTargetException; - +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; import java.net.InetSocketAddress; -import java.io.*; import javax.net.SocketFactory; -import org.apache.commons.logging.*; - -import org.apache.hadoop.io.*; +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.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.ObjectWritable; +import org.apache.hadoop.io.UTF8; +import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.ipc.Client.ConnectionId; import org.apache.hadoop.ipc.RPC.RpcInvoker; -import org.apache.hadoop.ipc.VersionedProtocol; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.Time; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.*; -import java.util.Collection; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; + +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** An RpcEngine implementation for Writable data. */ @InterfaceStability.Evolving public class WritableRpcEngine implements RpcEngine { + private static final XTrace.Logger xtrace = XTrace.getLogger(RPC.class); private static final Log LOG = LogFactory.getLog(RPC.class); //writableRpcVersion should be updated if there is a change @@ -231,8 +235,8 @@ public Object invoke(Object proxy, Method method, Object[] args) startTime = Time.now(); } - XTraceContext.logEvent(RPC.class, "WritableRpcEngine", "RPC Client invoking remote method "+method.getName(), "ConnectionID", this.remoteId); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("RPC Client invoking remote method "+method.getName(), "ConnectionID", this.remoteId); + Context start_context = XTrace.get(); try { // xtrace try ObjectWritable value = (ObjectWritable) @@ -242,14 +246,14 @@ public Object invoke(Object proxy, Method method, Object[] args) LOG.debug("Call: " + method.getName() + " " + callTime); } - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(RPC.class, "WritableRpcEngine", "Client invocation of "+method.getName()+" successful"); + XTrace.join(start_context); + xtrace.log("Client invocation of "+method.getName()+" successful"); return value.get(); } catch (Exception e) {// xtrace catch - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(RPC.class, "WritableRpcEngine", "Remote invocation of "+method.getName()+" failed due to exception: "+e.getClass().getName(), "Message", e.getMessage()); + XTrace.join(start_context); + xtrace.log("Remote invocation of "+method.getName()+" failed due to exception: "+e.getClass().getName(), "Message", e.getMessage()); throw e; } } @@ -438,8 +442,8 @@ public Writable call(org.apache.hadoop.ipc.RPC.Server server, Invocation call = (Invocation)rpcRequest; if (server.verbose) log("Call: " + call); - XTraceContext.logEvent(WritableRpcInvoker.class, "WritableRpcEngine", "Invoking method "+call.getMethodName()); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("Invoking method", "Method", call.getMethodName()); + Context start_context = XTrace.get(); try { // xtrace try // Verify writable rpc version @@ -514,9 +518,8 @@ public Writable call(org.apache.hadoop.ipc.RPC.Server server, processingTime); if (server.verbose) log("Return: "+value); - - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(WritableRpcInvoker.class, "WritableRpcEngine", "Invocation of "+call.getMethodName()+" completed, responding to client"); + XTrace.join(start_context); + xtrace.log("Invocation of method completed, responding to client", "Method", call.getMethodName()); return new ObjectWritable(method.getReturnType(), value); @@ -539,8 +542,8 @@ public Writable call(org.apache.hadoop.ipc.RPC.Server server, } } catch (IOException e) { // xtrace catch - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(WritableRpcInvoker.class, "ProtoBufRpcInvoker", "Failed to invoke method "+call.getMethodName()+": "+e.getClass().getName(), "Message", e.getMessage()); + XTrace.join(start_context); + xtrace.log("Failed to invoke method "+call.getMethodName()+": "+e.getClass().getName(), "Message", e.getMessage()); throw e; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java index 7c94beaebe03..dc9738bc9afe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java @@ -22,6 +22,7 @@ import java.io.OutputStream; import java.lang.reflect.Constructor; import java.net.BindException; +import java.net.ConnectException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.NetworkInterface; @@ -34,12 +35,17 @@ import java.net.URI; import java.net.URISyntaxException; import java.net.UnknownHostException; -import java.net.ConnectException; import java.nio.channels.SocketChannel; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Map.Entry; -import java.util.regex.Pattern; -import java.util.*; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.regex.Pattern; import javax.net.SocketFactory; @@ -58,12 +64,13 @@ import com.google.common.base.Preconditions; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @InterfaceStability.Unstable public class NetUtils { + private static final XTrace.Logger xtrace = XTrace.getLogger(NetUtils.class); private static final Log LOG = LogFactory.getLog(NetUtils.class); private static Map hostToResolved = @@ -513,8 +520,8 @@ public static void connect(Socket socket, throw new IllegalArgumentException("Illegal argument for connect()"); } - XTraceContext.logEvent(NetUtils.class, "NetUtils", "Connecting to remote...", "Socket", socket.toString(), "Timeout", timeout); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("Connecting to remote", "Socket", socket.toString(), "Timeout", timeout); + Context start_context = XTrace.get(); try { // xtrace try SocketChannel ch = socket.getChannel(); @@ -554,11 +561,11 @@ public static void connect(Socket socket, "No daemon is listening on the target port."); } - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(NetUtils.class, "NetUtils", "Connected to remote host"); + XTrace.join(start_context); + xtrace.log("Connected to remote host"); } catch (IOException e) { // xtrace catch - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(NetUtils.class, "NetUtils", "Failed to connect to remote host: "+e.getClass().getName(), "Message", e.getMessage()); + XTrace.join(start_context); + xtrace.log("Failed to connect to remote host: "+e.getClass().getName(), "Message", e.getMessage()); throw e; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java index 67ea33ad2cb1..c062459c9d66 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java @@ -24,12 +24,14 @@ import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto; import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.UserInformationProto; -import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*; +import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcKindProto; +import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto; import org.apache.hadoop.security.SaslRpcServer.AuthMethod; import org.apache.hadoop.security.UserGroupInformation; import com.google.protobuf.ByteString; -import edu.berkeley.xtrace.XTraceContext; + +import edu.brown.cs.systems.xtrace.XTrace; public abstract class ProtoUtil { @@ -166,9 +168,8 @@ public static RpcRequestHeaderProto makeRpcRequestHeader(RPC.RpcKind rpcKind, RpcRequestHeaderProto.Builder result = RpcRequestHeaderProto.newBuilder(); result.setRpcKind(convert(rpcKind)).setRpcOp(operation).setCallId(callId) .setRetryCount(retryCount).setClientId(ByteString.copyFrom(uuid)); - if (XTraceContext.isValid()) { - result.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); - } + if (XTrace.active()) + result.setXtrace(ByteString.copyFrom(XTrace.bytes())); return result.build(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java index f185dd0bb5eb..e826fbc91b96 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java @@ -32,9 +32,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - /** * A base class for running a Unix command. * @@ -387,13 +384,15 @@ private void runCommand() throws IOException { if (environment != null) { builder.environment().putAll(this.environment); } - // put xtrace context if there is one, merging as appropriate - XTraceMetadata child_process_xtrace = null; - if (XTraceContext.isValid()) { - child_process_xtrace = XTraceContext.startChildProcess(); - builder.environment().put(XTraceContext.XTRACE_CONTEXT_ENV_VARIABLE, XTraceContext.logMerge().toString()); - builder.environment().put(XTraceContext.XTRACE_SUBPROCESS_ENV_VARIABLE, child_process_xtrace.toString()); - } + +// // TODO: implement this in new xtrace +// // put xtrace context if there is one, merging as appropriate +// XTraceMetadata child_process_xtrace = null; +// if (XTraceContext.isValid()) { +// child_process_xtrace = XTraceContext.startChildProcess(); +// builder.environment().put(XTraceContext.XTRACE_CONTEXT_ENV_VARIABLE, XTraceContext.logMerge().toString()); +// builder.environment().put(XTraceContext.XTRACE_SUBPROCESS_ENV_VARIABLE, child_process_xtrace.toString()); +// } if (dir != null) { builder.directory(this.dir); @@ -457,8 +456,8 @@ public void run() { // wait for the process to finish and check the exit code exitCode = process.waitFor(); - // join the xtrace process - XTraceContext.joinChildProcess(child_process_xtrace); +// // TODO: join the xtrace process +// XTraceContext.joinChildProcess(child_process_xtrace); try { // make sure that the error thread exits errThread.join(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 77bdf8520f23..6e9af4390867 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -61,9 +61,7 @@ import java.io.OutputStream; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.NetworkInterface; import java.net.Socket; -import java.net.SocketException; import java.net.SocketAddress; import java.net.URI; import java.net.UnknownHostException; @@ -99,6 +97,7 @@ import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Options.ChecksumOpt; import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.VolumeId; import org.apache.hadoop.fs.permission.FsPermission; @@ -112,13 +111,13 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; +import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; @@ -131,8 +130,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; -import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.protocolPB.PBHelper; +import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; @@ -163,7 +162,6 @@ import com.google.common.base.Preconditions; import com.google.common.net.InetAddresses; -import edu.berkeley.xtrace.XTraceContext; /******************************************************** * DFSClient can connect to a Hadoop Filesystem and @@ -1020,15 +1018,12 @@ public LocatedBlocks getLocatedBlocks(String src, long start, long length) static LocatedBlocks callGetBlockLocations(ClientProtocol namenode, String src, long start, long length) throws IOException { - //XTraceContext.startTrace("DFSClient", "GetBlockLocations", "getBlockLocations", src+" ["+start+":"+(start+length)+"]"); try { return namenode.getBlockLocations(src, start, length); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, FileNotFoundException.class, UnresolvedPathException.class); - } finally { - XTraceContext.logEvent("DFSClient", "GetBlockLocations complete"); } } @@ -1520,7 +1515,6 @@ public boolean setReplication(String src, short replication) public boolean rename(String src, String dst) throws IOException { checkOpen(); try { - //XTraceContext.startTrace("DFSClient", "Rename", "rename", src+" -> "+dst); return namenode.rename(src, dst); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1528,8 +1522,6 @@ public boolean rename(String src, String dst) throws IOException { DSQuotaExceededException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); - } finally { - XTraceContext.logEvent("DFSClient", "Rename complete"); } } @@ -1540,14 +1532,11 @@ public boolean rename(String src, String dst) throws IOException { public void concat(String trg, String [] srcs) throws IOException { checkOpen(); try { - //XTraceContext.startTrace("DFSClient", "Concat", "concat"); namenode.concat(trg, srcs); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); - } finally { - XTraceContext.logEvent("DFSClient", "Concat complete"); } } /** @@ -1558,7 +1547,6 @@ public void rename(String src, String dst, Options.Rename... options) throws IOException { checkOpen(); try { - //XTraceContext.startTrace("DFSClient", "Rename", "rename", src+" -> "+dst); namenode.rename2(src, dst, options); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1570,8 +1558,6 @@ public void rename(String src, String dst, Options.Rename... options) NSQuotaExceededException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); - } finally { - XTraceContext.logEvent("DFSClient", "Rename complete"); } } /** @@ -1581,12 +1567,7 @@ public void rename(String src, String dst, Options.Rename... options) @Deprecated public boolean delete(String src) throws IOException { checkOpen(); - //XTraceContext.startTrace("DFSClient", "Delete", "delete", src); - try { // xtrace try return namenode.delete(src, true); - } finally { - XTraceContext.logEvent("DFSClient", "Delete complete"); - } } /** @@ -1599,7 +1580,6 @@ public boolean delete(String src) throws IOException { public boolean delete(String src, boolean recursive) throws IOException { checkOpen(); try { - //XTraceContext.startTrace("DFSClient", "Delete", "delete", src); return namenode.delete(src, recursive); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, @@ -1607,8 +1587,6 @@ public boolean delete(String src, boolean recursive) throws IOException { SafeModeException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); - } finally { - XTraceContext.logEvent("DFSClient", "Delete complete"); } } @@ -1642,14 +1620,11 @@ public DirectoryListing listPaths(String src, byte[] startAfter, throws IOException { checkOpen(); try { - //XTraceContext.startTrace("DFSClient", "ListPaths", "listPaths", src); return namenode.getListing(src, startAfter, needLocation); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, FileNotFoundException.class, UnresolvedPathException.class); - } finally { - XTraceContext.logEvent("DFSClient", "ListPaths complete"); } } @@ -1664,14 +1639,11 @@ public DirectoryListing listPaths(String src, byte[] startAfter, public HdfsFileStatus getFileInfo(String src) throws IOException { checkOpen(); try { - //XTraceContext.startTrace("DFSClient", "GetFileInfo", "getFileInfo", src); return namenode.getFileInfo(src); } catch(RemoteException re) { throw re.unwrapRemoteException(AccessControlException.class, FileNotFoundException.class, UnresolvedPathException.class); - } finally { - XTraceContext.logEvent("DFSClient", "GetFileInfo complete"); } } @@ -2150,12 +2122,9 @@ public String createSnapshot(String snapshotRoot, String snapshotName) throws IOException { checkOpen(); try { - //XTraceContext.startTrace("DFSClient", "CreateSnapshot", "createSnapshot", snapshotRoot); return namenode.createSnapshot(snapshotRoot, snapshotName); } catch(RemoteException re) { throw re.unwrapRemoteException(); - } finally { - XTraceContext.logEvent("DFSClient", "CreateSnapshot complete"); } } @@ -2171,12 +2140,9 @@ public String createSnapshot(String snapshotRoot, String snapshotName) public void deleteSnapshot(String snapshotRoot, String snapshotName) throws IOException { try { - //XTraceContext.startTrace("DFSClient", "DeleteSnapshot", "deleteSnapshot", snapshotRoot); namenode.deleteSnapshot(snapshotRoot, snapshotName); } catch(RemoteException re) { throw re.unwrapRemoteException(); - } finally { - XTraceContext.logEvent("DFSClient", "DeleteSnapshot complete"); } } @@ -2192,12 +2158,9 @@ public void renameSnapshot(String snapshotDir, String snapshotOldName, String snapshotNewName) throws IOException { checkOpen(); try { - //XTraceContext.startTrace("DFSClient", "RenameSnapshot", "renameSnapshot", snapshotOldName+" -> "+snapshotNewName); namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName); } catch(RemoteException re) { throw re.unwrapRemoteException(); - } finally { - XTraceContext.logEvent("DFSClient", "RenameSnapshot complete"); } } @@ -2383,7 +2346,6 @@ public boolean primitiveMkdir(String src, FsPermission absPermission, boolean createParent) throws IOException { checkOpen(); - //XTraceContext.startTrace("DFSClient", "MkDir", "mkdir", src); if (absPermission == null) { absPermission = FsPermission.getDefault().applyUMask(dfsClientConf.uMask); @@ -2405,8 +2367,6 @@ public boolean primitiveMkdir(String src, FsPermission absPermission, DSQuotaExceededException.class, UnresolvedPathException.class, SnapshotAccessControlException.class); - } finally { - XTraceContext.logEvent("DFSClient", "MkDir complete"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 850ce4a1d635..a6f3a1388fd9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -33,7 +33,6 @@ import java.nio.channels.ClosedChannelException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.EnumSet; import java.util.LinkedList; import java.util.List; @@ -61,8 +60,8 @@ import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; -import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor; +import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; @@ -93,8 +92,9 @@ import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; + /**************************************************************** @@ -121,6 +121,7 @@ ****************************************************************/ @InterfaceAudience.Private public class DFSOutputStream extends FSOutputSummer implements Syncable { + private final XTrace.Logger xtrace = XTrace.getLogger(DFSOutputStream.class); private final DFSClient dfsClient; private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB private Socket s; @@ -153,7 +154,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable { private final short blockReplication; // replication factor of file private boolean shouldSyncBlock = false; // force blocks to disk upon close - private Collection lastAckedXTraceContext = null; + private Context lastAckedXTraceContext = null; private class Packet { long seqno; // sequencenumber of buffer in block @@ -186,7 +187,7 @@ private class Packet { private static final long HEART_BEAT_SEQNO = -1L; - private Collection xtrace = null; + private Context xtrace_context = null; /** * Create a heartbeat packet. @@ -228,11 +229,11 @@ private class Packet { } void joinXTraceContext() { - XTraceContext.joinContext(xtrace); + XTrace.join(xtrace_context); } void rememberXTraceContext() { - xtrace = XTraceContext.getThreadContext(); + xtrace_context = XTrace.get(); } void writeData(byte[] inarray, int off, int len) { @@ -444,7 +445,7 @@ private void setFavoredNodes(String[] favoredNodes) { private void initDataStreaming() { this.setName("DataStreamer for file " + src + " block " + block); - XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Kicking off response processor"); + xtrace.log("Kicking off response processor"); response = new ResponseProcessor(nodes); response.start(); stage = BlockConstructionStage.DATA_STREAMING; @@ -454,7 +455,7 @@ private void endBlock() { if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Closing old block " + block); } - XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Closing block", "BlockName", block.getBlockName()); + xtrace.log("Closing block", "BlockName", block.getBlockName()); this.setName("DataStreamer for file " + src); closeResponder(); closeStream(); @@ -514,7 +515,7 @@ public void run() { } // get packet to be sent. if (dataQueue.isEmpty()) { - XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Sending heartbeat packet"); + xtrace.log("Sending heartbeat packet"); one = new Packet(); // heartbeat packet } else { one = dataQueue.getFirst(); // regular data packet @@ -529,14 +530,14 @@ public void run() { if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Allocating new block"); } - XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Allocating new block"); + xtrace.log("Allocating new block"); nodes = nextBlockOutputStream(src); initDataStreaming(); } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) { if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Append to block " + block); } - XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Appending to block", "BlockName", block.getBlockName()); + xtrace.log("Appending to block", "BlockName", block.getBlockName()); setupPipelineForAppendOrRecovery(); initDataStreaming(); } @@ -568,7 +569,7 @@ public void run() { stage = BlockConstructionStage.PIPELINE_CLOSE; } - XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Sending packet"); + xtrace.log("Sending packet"); // send the packet synchronized (dataQueue) { @@ -598,7 +599,7 @@ public void run() { } lastPacket = Time.now(); - XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Packet sent"); + xtrace.log("Packet sent"); one.rememberXTraceContext(); if (one.isHeartbeatPacket()) { //heartbeat packet @@ -637,7 +638,7 @@ public void run() { } } catch (Throwable e) { DFSClient.LOG.warn("DataStreamer Exception", e); - XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "DataStreamerException", "Message", e.getMessage()); + xtrace.log("DataStreamerException", "Message", e.getMessage()); if (e instanceof IOException) { setLastException((IOException)e); } @@ -751,7 +752,7 @@ public void run() { long seqno = ack.getSeqno(); ack.joinXtraceContext(); - XTraceContext.logEvent(ResponseProcessor.class, "ResponseProcessor", "Processing ACK", "seqno", seqno); + xtrace.log("Processing ACK", "seqno", seqno); // processes response status from datanodes. for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) { final Status reply = ack.getReply(i); @@ -785,10 +786,10 @@ public void run() { // update bytesAcked block.setNumBytes(one.getLastByteOffsetBlock()); - XTraceContext.logEvent(ResponseProcessor.class, "ResponseProcessor", "Packet acknowledged"); + xtrace.log("Packet acknowledged"); synchronized (dataQueue) { lastAckedSeqno = seqno; - lastAckedXTraceContext = XTraceContext.getThreadContext(); + lastAckedXTraceContext = XTrace.get(); ackQueue.removeFirst(); dataQueue.notifyAll(); } @@ -804,7 +805,7 @@ public void run() { } DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception " + " for block " + block, e); - XTraceContext.logEvent(ResponseProcessor.class, "ResponseProcessor", "Exception processing responses", "Message", e.getMessage()); + xtrace.log("Exception processing responses", "Message", e.getMessage()); responderClosed = true; } } @@ -937,7 +938,7 @@ private void addDatanode2ExistingPipeline() throws IOException { * - Append/Create: * + no transfer, let NameNode replicates the block. */ - XTraceContext.logEvent(DataStreamer.class, "DataStreamer", "Adding DataNode to Existing Pipeline"); + xtrace.log("Adding DataNode to Existing Pipeline"); if (!isAppend && lastAckedSeqno < 0 && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) { //no data have been written @@ -1496,7 +1497,7 @@ private void queueCurrentPacket() { if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Queued packet " + currentPacket.seqno); } - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Packet added to output queue"); + xtrace.log("Packet added to output queue"); currentPacket.rememberXTraceContext(); currentPacket = null; dataQueue.notifyAll(); @@ -1509,8 +1510,8 @@ private void waitAndQueueCurrentPacket() throws IOException { boolean first = true; while (!closed && dataQueue.size() + ackQueue.size() > MAX_PACKETS) { if (first) { - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Waiting for packets to be acked", "lastAckedSeqno", lastAckedSeqno); - first = false; + xtrace.log("Waiting for packets to be acked", "lastAckedSeqno", lastAckedSeqno); + first = false; } try { dataQueue.wait(); @@ -1526,7 +1527,7 @@ private void waitAndQueueCurrentPacket() throws IOException { break; } if (dataQueue.size() + ackQueue.size() <= MAX_PACKETS) - XTraceContext.joinContext(lastAckedXTraceContext); + XTrace.join(lastAckedXTraceContext); } checkClosed(); queueCurrentPacket(); @@ -1564,7 +1565,7 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] che ", chunksPerPacket=" + chunksPerPacket + ", bytesCurBlock=" + bytesCurBlock); } - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Allocating new packet", "seqno", currentPacket.seqno); + xtrace.log("Allocating new packet", "seqno", currentPacket.seqno); currentPacket.rememberXTraceContext(); } @@ -1608,7 +1609,7 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] che // indicate the end of block and reset bytesCurBlock. // if (bytesCurBlock == blockSize) { - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Sending empty packet to indicate end of block"); + xtrace.log("Sending empty packet to indicate end of block"); currentPacket = new Packet(0, 0, bytesCurBlock); currentPacket.lastPacketInBlock = true; currentPacket.syncBlock = shouldSyncBlock; @@ -1677,9 +1678,9 @@ public void hsync(EnumSet syncFlags) throws IOException { private void flushOrSync(boolean isSync, EnumSet syncFlags) throws IOException { if (isSync) - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "hysnc - Syncing all data to datanode disks"); + xtrace.log("hysnc - Syncing all data to datanode disks"); else - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "hflush - Flushing all data to datanode buffers"); + xtrace.log("hflush - Flushing all data to datanode buffers"); dfsClient.checkOpen(); @@ -1789,7 +1790,7 @@ private void flushOrSync(boolean isSync, EnumSet syncFlags) throw interrupt; } catch (IOException e) { DFSClient.LOG.warn("Error while syncing", e); - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Error while syncing", "Message", e.getMessage()); + xtrace.log("Error while syncing", "Message", e.getMessage()); synchronized (this) { if (!closed) { lastException = new IOException("IOException flush:" + e); @@ -1850,13 +1851,13 @@ private void waitForAckedSeqno(long seqno) throws IOException { if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Waiting for ack for: " + seqno); } - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Waiting for ack", "seqno", seqno); + xtrace.log("Waiting for ack", "seqno", seqno); synchronized (dataQueue) { while (!closed) { checkClosed(); if (lastAckedSeqno >= seqno) { - XTraceContext.joinContext(lastAckedXTraceContext); - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Ack received, continuing", "lastAckedSeqno", lastAckedSeqno); + XTrace.join(lastAckedXTraceContext); + xtrace.log("Ack received, continuing", "lastAckedSeqno", lastAckedSeqno); break; } try { @@ -1921,7 +1922,7 @@ public synchronized void close() throws IOException { } try { - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Closing stream"); + xtrace.log("Closing stream"); flushBuffer(); // flush from all upper layers @@ -1944,7 +1945,7 @@ public synchronized void close() throws IOException { dfsClient.endFileLease(src); } finally { closed = true; - XTraceContext.logEvent(DFSOutputStream.class, "DFSOutputStream", "Stream closed"); + xtrace.log("Stream closed"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java index b175c4ffc866..aaf8ac2b50de 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java @@ -38,7 +38,7 @@ import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** *

    @@ -72,6 +72,7 @@ *

    */ class LeaseRenewer { + static final XTrace.Logger xtrace = XTrace.getLogger(LeaseRenewer.class); static final Log LOG = LogFactory.getLog(LeaseRenewer.class); static final long LEASE_RENEWER_GRACE_DEFAULT = 60*1000L; @@ -293,9 +294,8 @@ synchronized void put(final String src, final DFSOutputStream out, daemon = new Daemon(new Runnable() { @Override public void run() { - XTraceContext.logEvent(LeaseRenewer.class, "LeaseRenewer", "Lease renewer daemon for " + clientsString() - + " with renew id " + id + " started"); - XTraceContext.clearThreadContext(); // don't let the task id leak to lease renewer. + xtrace.log("Lease renewer daemon started"); + XTrace.stop(); // don't let the task id leak to lease renewer. // if lease renewer tracing desired, should start a new task here and maybe log an event. try { if (LOG.isDebugEnabled()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index 655782d7367d..a9e393fd0671 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -44,8 +44,6 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; -import edu.berkeley.xtrace.XTraceContext; - /** * @deprecated this is an old implementation that is being left around @@ -386,7 +384,6 @@ public static RemoteBlockReader newBlockReader(String file, DatanodeID datanodeID, PeerCache peerCache) throws IOException { - XTraceContext.logEvent(RemoteBlockReader.class, "RemoteBlockReader", "Reading remote block using unsupported block reader", "file", file, "BlockName", block.getBlockName()); // in and out will be closed when sock is closed (by the caller) final DataOutputStream out = diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java index bea61ce3ece3..736e7d4e65d6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java @@ -38,10 +38,8 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; -import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; @@ -51,7 +49,7 @@ import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** * This is a wrapper around connection to datanode @@ -82,6 +80,7 @@ @InterfaceAudience.Private public class RemoteBlockReader2 implements BlockReader { + static final XTrace.Logger xtrace = XTrace.getLogger(RemoteBlockReader2.class); static final Log LOG = LogFactory.getLog(RemoteBlockReader2.class); final private Peer peer; @@ -216,7 +215,7 @@ private void readNextPacket() throws IOException { // If we've now satisfied the whole client read, read one last packet // header, which should be empty if (bytesNeededToFinish <= 0) { - XTraceContext.logEvent(RemoteBlockReader2.class, "RemoteBlockReader2", "Block finished, reading trailing empty packet"); + xtrace.log("Block finished, reading trailing empty packet"); readTrailingEmptyPacket(); if (verifyChecksum) { sendReadResult(Status.CHECKSUM_OK); @@ -380,7 +379,7 @@ public static BlockReader newBlockReader(String file, String clientName, Peer peer, DatanodeID datanodeID, PeerCache peerCache) throws IOException { - XTraceContext.logEvent(RemoteBlockReader2.class, "RemoteBlockReader2", "Reading remote block", "file", file, "BlockName", block.getBlockName()); + xtrace.log("Reading remote block", "file", file, "BlockName", block.getBlockName()); try { // xtrace try // in and out will be closed when sock is closed (by the caller) @@ -419,7 +418,7 @@ public static BlockReader newBlockReader(String file, datanodeID, peerCache); } catch (IOException e) { - XTraceContext.logEvent(RemoteBlockReader2.class, "RemoteBlockReader2", "IOException reading remote block", "Message", e.getMessage()); + xtrace.log("IOException reading remote block", "Message", e.getMessage()); throw e; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java index 3887958e4937..2275f9ba707d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java @@ -4,8 +4,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto; import com.google.protobuf.ByteString; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; + +import edu.brown.cs.systems.xtrace.XTrace; /** * Contains some utility functions for XTrace instrumentation. Saves having to repeat @@ -30,8 +30,8 @@ public static BlockOpResponseProto.Builder newBlockOpResponseProtoBuilder() { * @param builder */ public static void setXtrace(BlockOpResponseProto.Builder builder) { - if (XTraceContext.isValid()) - builder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + if (XTrace.active()) + builder.setXtrace(ByteString.copyFrom(XTrace.bytes())); } /** @@ -39,13 +39,8 @@ public static void setXtrace(BlockOpResponseProto.Builder builder) { * @param p */ public static void join(BlockOpResponseProto p) { - if (!p.hasXtrace()) - return; - - ByteString xbs = p.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); - if (xmd.isValid()) - XTraceContext.joinContext(xmd); + if (p.hasXtrace()) + XTrace.join(p.getXtrace().toByteArray()); } /** @@ -63,8 +58,8 @@ public static ClientReadStatusProto.Builder newClientReadStatusProtoBuilder() { * @param builder */ public static void setXtrace(ClientReadStatusProto.Builder builder) { - if (XTraceContext.isValid()) - builder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + if (XTrace.active()) + builder.setXtrace(ByteString.copyFrom(XTrace.bytes())); } /** @@ -72,13 +67,8 @@ public static void setXtrace(ClientReadStatusProto.Builder builder) { * @param p */ public static void join(ClientReadStatusProto p) { - if (!p.hasXtrace()) - return; - - ByteString xbs = p.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); - if (xmd.isValid()) - XTraceContext.joinContext(xmd); + if (p.hasXtrace()) + XTrace.join(p.getXtrace().toByteArray()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java index fa5537d16a5a..d08d2158957f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java @@ -31,8 +31,10 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; + import com.google.protobuf.ByteString; -import edu.berkeley.xtrace.XTraceContext; + +import edu.brown.cs.systems.xtrace.XTrace; /** @@ -84,8 +86,8 @@ static BaseHeaderProto buildBaseHeader(ExtendedBlock blk, Builder header = BaseHeaderProto.newBuilder() .setBlock(PBHelper.convert(blk)) .setToken(PBHelper.convert(blockToken)); - if (XTraceContext.isValid()) - header.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + if (XTrace.active()) + header.setXtrace(ByteString.copyFrom(XTrace.bytes())); return header.build(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java index 9d97b2cbb835..fe4dfd0b77b7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java @@ -28,15 +28,12 @@ import org.apache.hadoop.hdfs.util.ByteBufferOutputStream; import com.google.common.base.Preconditions; -import com.google.common.primitives.Shorts; import com.google.common.primitives.Ints; -import com.google.protobuf.InvalidProtocolBufferException; +import com.google.common.primitives.Shorts; import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; -import edu.berkeley.xtrace.OptionField; -import edu.berkeley.xtrace.TaskID; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; /** * Header data for each packet that goes through the read/write pipelines. @@ -57,18 +54,17 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class PacketHeader { + private static final XTrace.Logger xtrace = XTrace.getLogger(PacketHeader.class); private static final int MAX_PROTO_SIZE; static { - XTraceMetadata max = new XTraceMetadata(new TaskID(8), 0L); - max.addOption(new OptionField((byte)0, new byte[254])); - MAX_PROTO_SIZE = PacketHeaderProto.newBuilder() - .setOffsetInBlock(0) - .setSeqno(0) - .setLastPacketInBlock(false) - .setDataLen(0) - .setSyncBlock(false) - .setXtrace(ByteString.copyFrom(max.pack())) - .build().getSerializedSize(); + MAX_PROTO_SIZE = PacketHeaderProto.newBuilder() + .setOffsetInBlock(0) + .setSeqno(0) + .setLastPacketInBlock(false) + .setDataLen(0) + .setSyncBlock(false) + .setXtrace(ByteString.copyFrom(XTrace.XTRACE_BYTES_EXAMPLE)) + .build().getSerializedSize(); } public static final int PKT_LENGTHS_LEN = Ints.BYTES + Shorts.BYTES; @@ -94,8 +90,10 @@ public PacketHeader(int packetLen, long offsetInBlock, long seqno, .setLastPacketInBlock(lastPacketInBlock) .setDataLen(dataLen); - if (XTraceContext.isValid()) - builder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + if (XTrace.active()) { + xtrace.log("Constructing packet header"); + builder.setXtrace(ByteString.copyFrom(XTrace.bytesBounded())); + } if (syncBlock) { // Only set syncBlock if it is specified. @@ -138,9 +136,7 @@ public boolean hasXTraceContext() { public void joinXTraceContext() { ByteString xbs = proto.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); - if (xmd.isValid()) - XTraceContext.joinContext(xmd); + XTrace.join(xbs.toByteArray()); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java index b64e0507570c..05fd1e0b5efd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java @@ -23,7 +23,6 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; -import java.util.Collection; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -34,8 +33,8 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; + /** * Class to handle reading packets one-at-a-time from the wire. @@ -51,6 +50,7 @@ public class PacketReceiver implements Closeable { */ private static final int MAX_PACKET_SIZE = 16 * 1024 * 1024; + static XTrace.Logger xtrace = XTrace.getLogger(PacketReceiver.class); static Log LOG = LogFactory.getLog(PacketReceiver.class); private static final DirectBufferPool bufferPool = new DirectBufferPool(); @@ -131,7 +131,8 @@ private void doRead(ReadableByteChannel ch, InputStream in) // CHECKSUMS: the crcs for the data chunk. May be missing if // checksums were not requested // DATA the actual block data - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Reading packet"); + + xtrace.log("Reading packet"); try { // xtrace try Preconditions.checkState(curHeader == null || !curHeader.isLastPacketInBlock()); @@ -187,7 +188,7 @@ private void doRead(ReadableByteChannel ch, InputStream in) } curHeader.setFieldsFromData(payloadLen, headerBuf); curHeader.joinXTraceContext(); - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Finished reading packet"); + xtrace.log("Finished reading packet"); // Compute the sub-slices of the packet int checksumLen = dataPlusChecksumLen - curHeader.getDataLen(); @@ -200,7 +201,7 @@ private void doRead(ReadableByteChannel ch, InputStream in) reslicePacket(headerLen, checksumLen, curHeader.getDataLen()); } catch (IOException e) { // xtrace catch - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Exception reading packet", "Message", e.getMessage()); + xtrace.log("Exception reading packet", "Message", e.getMessage()); throw e; } } @@ -212,7 +213,7 @@ public void mirrorPacketTo(DataOutputStream mirrorOut) throws IOException { Preconditions.checkState(!useDirectBuffers, "Currently only supported for non-direct buffers"); - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Mirroring packet"); + xtrace.log("Mirroring packet"); try { // xtrace try updateHeaderXTrace(); @@ -220,9 +221,9 @@ public void mirrorPacketTo(DataOutputStream mirrorOut) throws IOException { curPacketBuf.arrayOffset(), curPacketBuf.remaining()); - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Packet mirrored successfully"); + xtrace.log("Packet mirrored successfully"); } catch (IOException e) { // xtrace catch - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Exception writing block to mirror", "Message", e.getMessage()); + xtrace.log("Exception writing block to mirror", "Message", e.getMessage()); } } @@ -232,7 +233,7 @@ public void mirrorPacketTo(DataOutputStream mirrorOut) throws IOException { private void updateHeaderXTrace() { // Only update context if there was a previous one, and we assume they have the exact // same length, so we can just drop in a new packet header. - if (XTraceContext.isValid() && curHeader.hasXTraceContext()) { + if (XTrace.active() && curHeader.hasXTraceContext()) { PacketHeader newHeader = new PacketHeader(curHeader.getPacketLen(), curHeader.getOffsetInBlock(), curHeader.getSeqno(), curHeader.isLastPacketInBlock(), curHeader.getDataLen(), curHeader.getSyncBlock()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java index 14deeef2d008..34a9097e7501 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java @@ -34,14 +34,14 @@ import com.google.protobuf.ByteString; import com.google.protobuf.TextFormat; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +import edu.brown.cs.systems.xtrace.XTrace; /** Pipeline Acknowledgment **/ @InterfaceAudience.Private @InterfaceStability.Evolving public class PipelineAck { + + private static final XTrace.Logger xtrace = XTrace.getLogger(PipelineAck.class); PipelineAckProto proto; public final static long UNKOWN_SEQNO = -2; @@ -69,8 +69,10 @@ public PipelineAck(long seqno, Status[] replies, long downstreamAckTimeNanos) { .setSeqno(seqno) .addAllStatus(Arrays.asList(replies)) .setDownstreamAckTimeNanos(downstreamAckTimeNanos); - if (XTraceContext.isValid()) - builder.setXtrace(ByteString.copyFrom(XTraceContext.logMerge().pack())); + if (XTrace.active()) { + xtrace.log("creating pipelined ack"); + builder.setXtrace(ByteString.copyFrom(XTrace.bytes())); + } proto = builder.build(); } @@ -111,13 +113,8 @@ public boolean hasXtrace() { } public void joinXtraceContext() { - if (proto.hasXtrace()) { - ByteString xbs = proto.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), - 0, xbs.size()); - if (xmd.isValid()) - XTraceContext.joinContext(xmd); - } + if (proto.hasXtrace()) + XTrace.join(proto.getXtrace().toByteArray()); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index a950b177e24e..e131588a2b13 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hdfs.protocol.datatransfer; -import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto; +import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; import java.io.DataInputStream; import java.io.IOException; @@ -30,13 +30,12 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; -import com.google.protobuf.ByteString; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; + +import edu.brown.cs.systems.xtrace.XTrace; /** Receiver */ @InterfaceAudience.Private @@ -90,12 +89,8 @@ protected final void processOp(Op op) throws IOException { } private static void XTraceJoinStart(String type, BaseHeaderProto header) { - if (header!=null && header.hasXtrace()) { - ByteString xbs = header.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); - if (xmd!=null && xmd.isValid()) - XTraceContext.setThreadContext(xmd); - } + if (header!=null && header.hasXtrace()) + XTrace.join(header.getXtrace().toByteArray()); } /** Receive OP_READ_BLOCK */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index b931daab17c0..447ea2e7d5f4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -58,7 +58,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Maps; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** * A thread per active or standby namenode to perform: @@ -72,6 +72,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; @@ -315,7 +316,7 @@ void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) { pendingIncrementalBR.put( bInfo.getBlock().getBlockId(), bInfo); pendingReceivedRequests++; - XTraceContext.logEvent(BPServiceActor.class, "BPServiceActor", "Triggering block report to namenode"); + xtrace.log("Triggering block report to namenode"); pendingIncrementalBR.notifyAll(); } } 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 c11edfbdc0bd..c47e46fa291c 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 @@ -29,7 +29,6 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collection; import java.util.LinkedList; import java.util.zip.Checksum; @@ -54,14 +53,15 @@ import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.StringUtils; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +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 Log LOG = DataNode.LOG; static final Log ClientTraceLog = DataNode.ClientTraceLog; @@ -269,7 +269,7 @@ public void close() throws IOException { // close checksum file try { if (checksumOut != null) { - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushing checksum file"); + xtrace.log("Closing checksum file"); long flushStartNanos = System.nanoTime(); checksumOut.flush(); long flushEndNanos = System.nanoTime(); @@ -282,7 +282,7 @@ public void close() throws IOException { measuredFlushTime = true; checksumOut.close(); checksumOut = null; - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Checksum file closed"); + xtrace.log("BlockReceiver", "Checksum file closed"); } } catch(IOException e) { ioe = e; @@ -293,7 +293,7 @@ public void close() throws IOException { // close block file try { if (out != null) { - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushing block file"); + xtrace.log("Closing block file"); long flushStartNanos = System.nanoTime(); out.flush(); long flushEndNanos = System.nanoTime(); @@ -306,7 +306,7 @@ public void close() throws IOException { measuredFlushTime = true; out.close(); out = null; - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Block file closed"); + xtrace.log("Block file closed"); } } catch (IOException e) { ioe = e; @@ -345,7 +345,7 @@ void flushOrSync(boolean isSync) throws IOException { long flushStartNanos = System.nanoTime(); out.flush(); long flushEndNanos = System.nanoTime(); - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Flushed file output stream"); + xtrace.log("Flushed file output stream"); if (isSync && (out instanceof FileOutputStream)) { long fsyncStartNanos = flushEndNanos; ((FileOutputStream)out).getChannel().force(true); @@ -569,7 +569,7 @@ private int receivePacket() throws IOException { int numBytesToDisk = (int)(offsetInBlock-onDiskLen); // Write data to disk. - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Writing packet to file output stream", "start", startByteToDisk, "size", numBytesToDisk); + 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 @@ -661,7 +661,7 @@ void receiveBlock( String mirrAddr, DataTransferThrottler throttlerArg, DatanodeInfo[] downstreams) throws IOException { - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Receiving Block"); + xtrace.log("Receiving Block"); try { // xtrace try syncOnClose = datanode.getDnConf().syncOnClose; @@ -736,9 +736,9 @@ void receiveBlock( responder = null; } } - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "Finished receiving block"); + xtrace.log("Finished receiving block"); } catch (IOException e) { // xtrace catch - XTraceContext.logEvent(BlockReceiver.class, "BlockReceiver", "IOException receiving block", "Message", e.getMessage()); + xtrace.log("IOException receiving block", "Message", e.getMessage()); } } @@ -949,7 +949,7 @@ public void run() { } seqno = ack.getSeqno(); - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Received ack", "seqno", seqno); + xtrace.log("Received ack", "seqno", seqno); } if (seqno != PipelineAck.UNKOWN_SEQNO || type == PacketResponderType.LAST_IN_PIPELINE) { @@ -1003,7 +1003,7 @@ public void run() { // and wait for the client to shut down the pipeline mirrorError = true; LOG.info(myString, ioe); - XTraceContext.logEvent(PacketResponder.class, "PacketResponder", "IOException from mirror", "Message", ioe.getMessage()); + xtrace.log("IOException from mirror", "Message", ioe.getMessage()); } } @@ -1016,7 +1016,7 @@ public void run() { * will detect that this datanode is bad, and rightly so. */ LOG.info(myString + ": Thread is interrupted."); - XTraceContext.logEvent(PacketResponder.class, "PacketResponder", "Thread is interrupted"); + xtrace.log("Thread is interrupted"); running = false; continue; } @@ -1069,7 +1069,7 @@ public void run() { continue; } } - XTraceContext.logEvent(PacketReceiver.class, "PacketResponder", "Acknowledging packet", "seqno", expected, "AckTimeNanos", totalAckTimeNanos); + xtrace.log("Acknowledging packet", "seqno", expected, "AckTimeNanos", totalAckTimeNanos); PipelineAck replyAck = new PipelineAck(expected, replies, totalAckTimeNanos); if (replyAck.isSuccess() && @@ -1087,7 +1087,7 @@ public void run() { removeAckHead(); // update bytes acked } - XTraceContext.logEvent(PacketReceiver.class, "PacketReceiver", "Packet ack sent."); + xtrace.log("Packet ack sent."); // terminate after sending response if this node detected // a checksum error if (myStatus == Status.ERROR_CHECKSUM) { @@ -1098,7 +1098,7 @@ public void run() { } } catch (IOException e) { LOG.warn("IOException in BlockReceiver.run(): ", e); - XTraceContext.logEvent(PacketReceiver.class, "PacketResponder", "IOException in BlockReceiver.run", "Message", e.getMessage()); + xtrace.log("IOException in BlockReceiver.run", "Message", e.getMessage()); if (running) { try { datanode.checkDiskError(e); // may throw an exception here @@ -1144,7 +1144,7 @@ private static class Packet { final Status ackStatus; /** Remember the XTrace context when this packet was created */ - private final Collection xtrace = XTraceContext.getThreadContext(); + private final Context xtrace_context = XTrace.get(); Packet(long seqno, boolean lastPacketInBlock, long offsetInBlock, long ackEnqueueNanoTime, Status ackStatus) { @@ -1156,7 +1156,7 @@ private static class Packet { } public void joinXtraceContext() { - XTraceContext.joinContext(xtrace); + XTrace.join(xtrace_context); } @Override 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 d59333b3394e..c1c67cd8934b 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,7 +47,7 @@ import com.google.common.base.Preconditions; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** * Reads a block from the disk and sends it to a recipient. @@ -90,6 +90,7 @@ * 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 Log LOG = DataNode.LOG; static final Log ClientTraceLog = DataNode.ClientTraceLog; private static final boolean is32Bit = @@ -228,9 +229,9 @@ class BlockSender implements java.io.Closeable { */ DataChecksum csum = null; if (verifyChecksum || sendChecksum) { - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Getting metadata input stream"); + xtrace.log("Getting metadata input stream"); final InputStream metaIn = datanode.data.getMetaDataInputStream(block); - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Got metadata input stream"); + xtrace.log("Got metadata input stream"); if (!corruptChecksumOk || metaIn != null) { if (metaIn == null) { //need checksum but meta-data not found @@ -458,7 +459,7 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out, int packetLen = dataLen + checksumDataLen + 4; boolean lastDataPacket = offset + dataLen == endOffset && dataLen > 0; - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Sending packet", "dataLen", dataLen, "numChunks", numChunks, "packetLen", packetLen); + xtrace.log("Sending packet", "dataLen", dataLen, "numChunks", numChunks, "packetLen", packetLen); // The packet buffer is organized as follows: // _______HHHHCCCCD?D?D?D? @@ -521,7 +522,7 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out, // normal transfer out.write(buf, headerOff, dataOff + dataLen - headerOff); } - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Packet send complete"); + xtrace.log("Packet send complete"); } catch (IOException e) { if (e instanceof SocketTimeoutException) { /* @@ -530,7 +531,7 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out, * the socket open). */ LOG.info("exception: ", e); - XTraceContext.logEvent(BlockSender.class, "BlockSender", "SocketTimeoutException"); + 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 @@ -545,7 +546,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); } - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Exception", "Message", ioem); + xtrace.log("Exception", "Message", ioem); } throw ioeToSocketException(e); } @@ -639,7 +640,7 @@ long sendBlock(DataOutputStream out, OutputStream baseStream, throw new IOException( "out stream is null" ); } - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Sending Block", "BlockName", block.getBlockName()); + xtrace.log("Sending Block", "BlockName", block.getBlockName()); try { // xtrace try initialOffset = offset; @@ -701,9 +702,9 @@ long sendBlock(DataOutputStream out, OutputStream baseStream, } sentEntireByteRange = true; - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Entire block sent", "totalRead", totalRead, "initialOffset", initialOffset); + xtrace.log("Entire block sent", "totalRead", totalRead, "initialOffset", initialOffset); } else { - XTraceContext.logEvent(BlockSender.class, "BlockSender", "Block send interrupted", "totalRead", totalRead, "initialOffset", initialOffset); + xtrace.log("Block send interrupted", "totalRead", totalRead, "initialOffset", initialOffset); } } finally { if (clientTraceFmt != null) { @@ -716,7 +717,7 @@ long sendBlock(DataOutputStream out, OutputStream baseStream, return totalRead; } catch (IOException e) { // xtrace catch - XTraceContext.logEvent(BlockSender.class, "BlockSender", "IOException sending block", "Message", e.getMessage()); + 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/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index ea016413df0c..4a58e4258167 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,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.BlockingService; +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 +221,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{ 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 eb4934a8defc..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; @@ -48,9 +48,9 @@ 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; @@ -77,13 +77,14 @@ import com.google.protobuf.ByteString; -import edu.berkeley.xtrace.XTraceContext; +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; @@ -223,14 +224,15 @@ public void run() { opStartTime = now(); processOp(op); ++opsProcessed; - XTraceContext.clearThreadContext(); + 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); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Error processing", "Message", t.getMessage()); + + xtrace.log("Error processing", "Message", t.getMessage()); } finally { if (LOG.isDebugEnabled()) { LOG.debug(datanode.getDisplayName() + ":Number of active connections is: " @@ -246,7 +248,7 @@ public void run() { public void requestShortCircuitFds(final ExtendedBlock blk, final Token token, int maxVersion) throws IOException { - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode RequestShortCircuitFds"); + xtrace.log("DataNode RequestShortCircuitFds"); updateCurrentThreadName("Passing file descriptors for block " + blk); BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder(); @@ -308,7 +310,7 @@ public void readBlock(final ExtendedBlock block, final long blockOffset, final long length, final boolean sendChecksum) throws IOException { - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode ReadBlock"); + xtrace.log("DataNode ReadBlock"); previousOpClientName = clientName; OutputStream baseStream = getOutputStream(); @@ -330,7 +332,7 @@ public void readBlock(final ExtendedBlock block, remoteAddress; updateCurrentThreadName("Sending block " + block); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Creating BlockSender"); + xtrace.log("Creating BlockSender"); try { try { blockSender = new BlockSender(block, blockOffset, length, @@ -342,15 +344,15 @@ public void readBlock(final ExtendedBlock block, throw e; } - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "WritingRPCResult"); + xtrace.log("Writing RPC result"); // send op status writeSuccessWithChecksumInfo(blockSender, new DataOutputStream(getOutputStream())); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sending Block"); + xtrace.log("Sending Block"); long read = blockSender.sendBlock(out, baseStream, null); // send data - - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sent Block"); + 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. @@ -368,10 +370,10 @@ public void readBlock(final ExtendedBlock block, LOG.debug("Error reading client status response. Will close connection.", ioe); IOUtils.closeStream(out); } - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Received RPC status enum"); + xtrace.log("Received RPC status"); } else { IOUtils.closeStream(out); - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Closed Stream"); + xtrace.log("Closed Stream"); } datanode.metrics.incrBytesRead((int) read); datanode.metrics.incrBlocksRead(); @@ -393,7 +395,7 @@ public void readBlock(final ExtendedBlock block, } finally { IOUtils.closeStream(blockSender); - XTraceContext.logEvent(DataXceiver.class, "DataNode", "ReadBlock Complete"); + xtrace.log("ReadBlock Complete"); } //update metrics @@ -413,7 +415,7 @@ public void writeBlock(final ExtendedBlock block, final long maxBytesRcvd, final long latestGenerationStamp, DataChecksum requestedChecksum) throws IOException { - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode WriteBlock"); + xtrace.log("DataNode WriteBlock"); previousOpClientName = clientname; updateCurrentThreadName("Receiving block " + block); final boolean isDatanode = clientname.length() == 0; @@ -487,7 +489,7 @@ public void writeBlock(final ExtendedBlock block, if (LOG.isDebugEnabled()) { LOG.debug("Connecting to datanode " + mirrorNode); } - XTraceContext.logEvent(DataXceiver.class, "DataXceiver writeBlock", "Connecting to mirror node", "mirrorNode", mirrorNode); + xtrace.log("Connecting to mirror node", "mirrorNode", mirrorNode); mirrorTarget = NetUtils.createSocketAddr(mirrorNode); mirrorSock = datanode.newSocket(); try { @@ -535,11 +537,11 @@ public void writeBlock(final ExtendedBlock block, " from downstream datanode with firstbadlink as " + firstBadLink); } - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Got response for connect ack from downstream datanode"); + xtrace.log("Got response for connect ack from downstream datanode"); } } catch (IOException e) { - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Exception transferring block"); + xtrace.log("Exception transferring block"); if (isClient) { XTraceProtoUtils.newBlockOpResponseProtoBuilder() .setStatus(ERROR) @@ -569,7 +571,7 @@ public void writeBlock(final ExtendedBlock block, // send connect-ack to source for clients and not transfer-RBW/Finalized if (isClient && !isTransfer) { - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Forwarding connect ack to upstream"); + xtrace.log("Forwarding connect ack to upstream"); if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) { LOG.info("Datanode " + targets.length + " forwarding connect ack to upstream firstbadlink is " + @@ -594,7 +596,7 @@ public void writeBlock(final ExtendedBlock block, if (LOG.isTraceEnabled()) { LOG.trace("TRANSFER: send close-ack"); } - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "Sending close ack"); + xtrace.log("Sending close ack"); writeResponse(SUCCESS, null, replyOut); } } @@ -628,7 +630,7 @@ public void writeBlock(final ExtendedBlock block, IOUtils.closeSocket(mirrorSock); IOUtils.closeStream(blockReceiver); - XTraceContext.logEvent(DataXceiver.class, "DataNode", "WriteBlock Complete"); + xtrace.log("WriteBlock Complete"); } //update metrics @@ -641,7 +643,7 @@ public void transferBlock(final ExtendedBlock blk, final Token blockToken, final String clientName, final DatanodeInfo[] targets) throws IOException { - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode TransferBlock"); + xtrace.log("DataNode TransferBlock"); checkAccess(socketOut, true, blk, blockToken, Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY); previousOpClientName = clientName; @@ -660,7 +662,7 @@ public void transferBlock(final ExtendedBlock blk, @Override public void blockChecksum(final ExtendedBlock block, final Token blockToken) throws IOException { - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode BlockChecksum"); + xtrace.log("DataNode BlockChecksum"); final DataOutputStream out = new DataOutputStream( getOutputStream()); checkAccess(out, true, block, blockToken, @@ -713,7 +715,7 @@ public void blockChecksum(final ExtendedBlock block, @Override public void copyBlock(final ExtendedBlock block, final Token blockToken) throws IOException { - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode CopyBlock"); + xtrace.log("DataNode CopyBlock"); updateCurrentThreadName("Copying block " + block); // Read in the header if (datanode.isBlockTokenEnabled) { @@ -789,7 +791,7 @@ public void replaceBlock(final ExtendedBlock block, final Token blockToken, final String delHint, final DatanodeInfo proxySource) throws IOException { - XTraceContext.logEvent(DataXceiver.class, "DataXceiver", "DataNode ReplaceBlock"); + xtrace.log("DataNode ReplaceBlock"); updateCurrentThreadName("Replacing block " + block + " from " + delHint); /* read header */ 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 4ef57d6bcde8..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 @@ -20,7 +20,6 @@ import java.io.File; import java.io.IOException; -import java.util.Collection; import java.util.HashSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -50,12 +49,10 @@ 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.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; /** * Runs the container task locally in a thread. @@ -176,7 +173,7 @@ 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()) { - XTraceContext.clearThreadContext(); + XTrace.stop(); try { event = eventQueue.take(); } catch (InterruptedException e) { // mostly via T_KILL? JOB_KILL? @@ -247,8 +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)); - XTraceContext.logEvent(LocalContainerLauncher.class, "LocalContainerLauncher", "Whoops. Fatal error."); - XTraceContext.joinParentProcess(); + // 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 35121dac4948..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,7 +55,7 @@ import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** * This class is responsible for talking to the task umblical. @@ -71,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; @@ -439,8 +440,7 @@ public JvmTask getTask(JvmContext context) throws IOException { task.joinContext(); launchedJVMs.remove(wJvmID); LOG.info("JVM with ID: " + jvmId + " given task: " + task.getTaskID()); - XTraceContext.logEvent(TaskAttemptListenerImpl.class, "TaskUmbilical getTask", - "Sending task to JVM", "Task ID", task.getTaskID(), "JVM ID", jvmId); + xtrace.log("Sending task to JVM", "Task ID", task.getTaskID(), "JVM ID", jvmId); jvmTask = new JvmTask(task, false); } } @@ -457,8 +457,7 @@ public void registerPendingTask( jvmIDToActiveAttemptMap.put(jvmID, task); task.rememberContext(); - XTraceContext.logEvent(TaskAttemptListenerImpl.class, "TaskUmbilical registerPendingTask", - "Task registered for JVM", "Task ID", task.getTaskID(), "JVM ID", jvmID); + 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 058b5abe9399..c0bbdcdbd3e2 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,19 +62,20 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.log4j.LogManager; -import edu.berkeley.xtrace.XTraceContext; +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 { - XTraceContext.logEvent(YarnChild.class, "YarnChild", "YarnChild starting"); + xtrace.log("YarnChild starting"); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); LOG.debug("Child starting"); @@ -136,7 +137,7 @@ public TaskUmbilicalProtocol run() throws Exception { myTask = umbilical.getTask(context); } if (myTask.shouldDie()) { - XTraceContext.logEvent(YarnChild.class, "YarnChild", "Task has been instructed to die."); + xtrace.log("Task has been instructed to die."); return; } @@ -163,19 +164,17 @@ public TaskUmbilicalProtocol run() throws Exception { public Object run() throws Exception { // use job-specified working directory FileSystem.get(job).setWorkingDirectory(job.getWorkingDirectory()); - XTraceContext.logEvent(YarnChild.class, "YarnChild", "Running task"); + xtrace.log("Running task"); taskFinal.run(job, umbilical); // run the task return null; } }); } catch (FSError e) { - XTraceContext.logEvent(YarnChild.class, "YarnChild FSError", "FSError from child: "+e.getClass().getName(), - "Message", e.getMessage()); + 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) { - XTraceContext.logEvent(YarnChild.class, "YarnChild Exception", "Exception running child: "+exception.getClass().getName(), - "Message", exception.getMessage()); + xtrace.log("Exception running child", "Exception", exception.getClass().getName(), "Message", exception.getMessage()); LOG.warn("Exception running child : " + StringUtils.stringifyException(exception)); try { @@ -202,8 +201,7 @@ public Object run() throws Exception { umbilical.fatalError(taskid, StringUtils.stringifyException(exception)); } } catch (Throwable throwable) { - XTraceContext.logEvent(YarnChild.class, "YarnChild Error", "Error running child: "+throwable.getClass().getName(), - "Message", throwable.getMessage()); + xtrace.log("Error running child", "Throwable", throwable.getClass().getName(), "Message", throwable.getMessage()); LOG.fatal("Error running child : " + StringUtils.stringifyException(throwable)); if (taskid != null) { @@ -214,8 +212,9 @@ public Object run() throws Exception { umbilical.fatalError(taskid, cause); } } finally { - XTraceContext.logEvent(YarnChild.class,"YarnChild", "YarnChild exiting"); - XTraceContext.joinParentProcess(); + xtrace.log("YarnChild exiting"); + // 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 d910484d2c44..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 @@ -19,7 +19,6 @@ package org.apache.hadoop.mapreduce.jobhistory; import java.io.IOException; -import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; @@ -57,8 +56,7 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; /** * The job history events get routed to this class. This class writes the Job @@ -263,7 +261,7 @@ public void run() { eventCounter++; } - XTraceContext.clearThreadContext(); + XTrace.stop(); try { event = eventQueue.take(); } catch (InterruptedException e) { 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 22c877572e63..f9ca2024780a 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 @@ -25,7 +25,6 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; -import java.util.Collection; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -135,8 +134,8 @@ import org.apache.hadoop.yarn.util.SystemClock; import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; + +import edu.brown.cs.systems.xtrace.XTrace; /** * The Map-Reduce Application Master. @@ -159,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); /** @@ -258,7 +258,7 @@ protected void serviceInit(final Configuration conf) throws Exception { /* 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 */ - XTraceContext.logEvent(MRAppMaster.class, "MRAppMaster", "Starting job", "Job ID", jobId); + xtrace.log("Starting job", "Job ID", jobId); int numReduceTasks = conf.getInt(MRJobConfig.NUM_REDUCES, 0); if ((numReduceTasks > 0 && @@ -512,7 +512,8 @@ public void cleanupStagingDir() throws IOException { * Exit call. Just in a function call to enable testing. */ protected void sysexit() { - XTraceContext.joinParentProcess(); + // TODO: FIX THIS XTRACE +// XTraceContext.joinParentProcess(); System.exit(0); } @@ -523,7 +524,7 @@ public void shutDownJob() { // note in a workflow scenario, this may lead to creation of a new // job (FIXME?) // Send job-end notification - XTraceContext.logEvent(JobFinishEvent.class, "JobFinish", "Handling Job Finished Event"); + 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 : " @@ -566,7 +567,7 @@ public void shutDownJob() { //Bring the process down by force. //Not needed after HADOOP-7140 LOG.info("Exiting MR AppMaster..GoodBye!"); - XTraceContext.logEvent(MRAppMaster.class, "MRAppMaster", "Application Master exiting"); + xtrace.log("Application Master exiting"); sysexit(); } @@ -1292,7 +1293,7 @@ private static void validateInputParam(String value, String param) public static void main(String[] args) { try { - XTraceContext.startTrace("MRAppMaster", "Application Master Launching"); + xtrace.log("Application Master Launching"); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); String containerIdStr = System.getenv(Environment.CONTAINER_ID.name()); @@ -1342,8 +1343,9 @@ public static void main(String[] args) { initAndStartAppMaster(appMaster, conf, jobUserName); } catch (Throwable t) { LOG.fatal("Error starting MRAppMaster", t); - XTraceContext.logEvent(MRAppMaster.class, "MRAppMaster", "Error starting MRAppMaster: "+t.getClass().getName(), "Message", t.getMessage()); - XTraceContext.joinParentProcess(); + xtrace.log("Error starting MRAppMaster: "+t.getClass().getName(), "Message", t.getMessage()); + // TODO: xtrace fix this +// XTraceContext.joinParentProcess(); System.exit(1); } } @@ -1352,17 +1354,13 @@ public static void main(String[] args) { // close of the JVM. static class MRAppMasterShutdownHook implements Runnable { MRAppMaster appMaster; - private Collection xtrace_context; MRAppMasterShutdownHook(MRAppMaster appMaster) { this.appMaster = appMaster; - xtrace_context = XTraceContext.getThreadContext(); } public void run() { - XTraceContext.setThreadContext(xtrace_context); LOG.info("MRAppMaster received a signal. Signaling RMCommunicator and " + "JobHistoryEventHandler."); - XTraceContext.logEvent(MRAppMasterShutdownHook.class, "MRAppMasterShutdownHook", - "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 @@ -1372,8 +1370,7 @@ public void run() { } appMaster.notifyIsLastAMRetry(appMaster.isLastAMRetry); appMaster.stop(); - XTraceContext.logEvent(MRAppMaster.class,"MRAppMaster", "MRAppMaster stopping"); - XTraceContext.clearThreadContext(); + 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 b645342722ca..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 @@ -18,7 +18,6 @@ package org.apache.hadoop.mapreduce.v2.app; -import java.util.Collection; import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -36,8 +35,7 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.util.Clock; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; /** @@ -157,7 +155,7 @@ public void run() { + " Timed out after " + taskTimeOut / 1000 + " secs")); eventHandler.handle(new TaskAttemptEvent(entry.getKey(), TaskAttemptEventType.TA_TIMED_OUT)); - XTraceContext.clearThreadContext(); + 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/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 bed6447f6a41..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,9 +126,8 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.Clock; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +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. @@ -142,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 @@ -599,13 +600,13 @@ JobEventType.JOB_KILL, new KillTasksTransition()) private boolean isUber = false; private Credentials jobCredentials; - private Collection completedTaskContexts = new XTraceMetadataCollection(); - private Collection succeededMapTaskContexts = new XTraceMetadataCollection(); - private Collection succeededReduceTaskContexts = new XTraceMetadataCollection(); - private Collection failedMapTaskContexts = new XTraceMetadataCollection(); - private Collection failedReduceTaskContexts = new XTraceMetadataCollection(); - private Collection killedMapTaskContexts = new XTraceMetadataCollection(); - private Collection killedReduceTaskContexts = new XTraceMetadataCollection(); + 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; @@ -709,9 +710,12 @@ public int getCompletedMaps() { public int getAndJoinCompletedMaps() { readLock.lock(); try { - XTraceContext.joinContext(succeededMapTaskContexts); - XTraceContext.joinContext(failedMapTaskContexts); - XTraceContext.joinContext(killedMapTaskContexts); + 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(); @@ -734,9 +738,12 @@ public int getCompletedReduces() { public int getAndJoinCompletedReduces() { readLock.lock(); try { - XTraceContext.joinContext(succeededReduceTaskContexts); - XTraceContext.joinContext(failedReduceTaskContexts); - XTraceContext.joinContext(killedReduceTaskContexts); + 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 { @@ -1066,8 +1073,7 @@ protected JobStateInternal checkReadyForCommit() { joinStateMachineXTraceContext(); if (completedTaskCount == tasks.size() && currentState == JobStateInternal.RUNNING) { - XTraceContext.logEvent(JobImpl.class, "JobImpl checkReadyForCommit", "Job ready for commit.", - "Completed Tasks", tasks.size(), "Current State", currentState); + xtrace.log("Job ready for commit.", "Completed Tasks", tasks.size(), "Current State", currentState); eventHandler.handle(new CommitterJobCommitEvent(jobId, getJobContext())); return JobStateInternal.COMMITTING; } @@ -1423,7 +1429,7 @@ public JobStateInternal transition(JobImpl job, JobEvent event) { // create the Tasks but don't start them yet createMapTasks(job, inputLength, taskSplitMetaInfo); createReduceTasks(job); - XTraceContext.logEvent(JobImpl.class, "JobImpl", "Created map and reduce tasks"); + xtrace.log("Created map and reduce tasks"); job.metrics.endPreparingJob(job); return JobStateInternal.INITED; @@ -1481,11 +1487,10 @@ protected void setup(JobImpl job) throws IOException { private void createMapTasks(JobImpl job, long inputLength, TaskSplitMetaInfo[] splits) { - XTraceContext.logEvent(JobImpl.class, "JobImpl", "Creating Map Tasks", "Input Length", inputLength, - "Num Splits", splits.length); - Collection start_context = XTraceContext.getThreadContext(); + 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) { - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); TaskImpl task = new MapTaskImpl(job.jobId, i, job.eventHandler, @@ -1503,10 +1508,10 @@ private void createMapTasks(JobImpl job, long inputLength, } private void createReduceTasks(JobImpl job) { - XTraceContext.logEvent(JobImpl.class, "JobImpl", "Creating Reduce Tasks", "Num Reduces", job.numReduceTasks); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("Creating Reduce Tasks", "Num Reduces", job.numReduceTasks); + Context start_context = XTrace.get(); for (int i = 0; i < job.numReduceTasks; i++) { - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); TaskImpl task = new ReduceTaskImpl(job.jobId, i, job.eventHandler, @@ -1607,8 +1612,10 @@ public void transition(JobImpl job, JobEvent event) { private void unsuccessfulFinish(JobStateInternal finalState) { if (finishTime == 0) setFinishTime(); cleanupProgress = 1.0f; - XTraceContext.joinContext(succeededMapTaskContexts); - XTraceContext.joinContext(succeededReduceTaskContexts); + for (Context ctx : succeededMapTaskContexts) + XTrace.join(ctx); + for (Context ctx : succeededReduceTaskContexts) + XTrace.join(ctx); JobUnsuccessfulCompletionEvent unsuccessfulJobEvent = new JobUnsuccessfulCompletionEvent(oldJobId, finishTime, @@ -1637,10 +1644,14 @@ private static JobFinishedEvent createJobFinishedEvent(JobImpl job) { job.mayBeConstructFinalFullCounters(); - XTraceContext.joinContext(job.succeededMapTaskContexts); - XTraceContext.joinContext(job.succeededReduceTaskContexts); - XTraceContext.joinContext(job.failedMapTaskContexts); - XTraceContext.joinContext(job.failedReduceTaskContexts); + 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, @@ -1878,10 +1889,12 @@ protected JobStateInternal checkJobAfterTaskCompletion(JobImpl job) { "failedMaps:" + job.failedMapTaskCount + " failedReduces:" + job.failedReduceTaskCount; - XTraceContext.joinContext(job.failedMapTaskContexts); - XTraceContext.joinContext(job.failedReduceTaskContexts); - XTraceContext.logEvent(JobImpl.class, "JobImpl checkJobAfterTaskCompletion", "Job failed as tasks failed.", - "Failed Maps", job.failedMapTaskCount, "Failed Reduces", 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); @@ -1896,10 +1909,10 @@ protected JobStateInternal checkJobAfterTaskCompletion(JobImpl job) { private void taskSucceeded(JobImpl job, Task task) { if (task.getType() == TaskType.MAP) { - job.succeededMapTaskContexts = XTraceContext.getThreadContext(job.succeededMapTaskContexts); + job.succeededMapTaskContexts.add(XTrace.get()); job.succeededMapTaskCount++; } else { - job.succeededReduceTaskContexts = XTraceContext.getThreadContext(job.succeededReduceTaskContexts); + job.succeededReduceTaskContexts.add(XTrace.get()); job.succeededReduceTaskCount++; } job.metrics.completedTask(task); @@ -1907,10 +1920,10 @@ private void taskSucceeded(JobImpl job, Task task) { private void taskFailed(JobImpl job, Task task) { if (task.getType() == TaskType.MAP) { - job.failedMapTaskContexts = XTraceContext.getThreadContext(job.failedMapTaskContexts); + job.failedMapTaskContexts.add(XTrace.get()); job.failedMapTaskCount++; } else if (task.getType() == TaskType.REDUCE) { - job.failedReduceTaskContexts = XTraceContext.getThreadContext(job.failedReduceTaskContexts); + job.failedReduceTaskContexts.add(XTrace.get()); job.failedReduceTaskCount++; } job.addDiagnostic("Task failed " + task.getID()); @@ -1919,10 +1932,10 @@ private void taskFailed(JobImpl job, Task task) { private void taskKilled(JobImpl job, Task task) { if (task.getType() == TaskType.MAP) { - job.killedMapTaskContexts = XTraceContext.getThreadContext(job.killedMapTaskContexts); + job.killedMapTaskContexts.add(XTrace.get()); job.killedMapTaskCount++; } else if (task.getType() == TaskType.REDUCE) { - job.killedReduceTaskContexts = XTraceContext.getThreadContext(job.killedReduceTaskContexts); + job.killedReduceTaskContexts.add(XTrace.get()); job.killedReduceTaskCount++; } job.metrics.killedTask(task); @@ -1984,8 +1997,8 @@ private static class MapTaskRescheduledTransition implements @Override public void transition(JobImpl job, JobEvent event) { //succeeded map task is restarted back - job.succeededMapTaskContexts = XTraceContext.getThreadContext(job.succeededMapTaskContexts); - job.completedTaskContexts = XTraceContext.getThreadContext(job.completedTaskContexts); + 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/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 9a44e7b16e18..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 @@ -21,8 +21,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.security.PrivilegedAction; -import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -60,8 +58,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; /** * This class is responsible for launching of containers. @@ -69,6 +66,7 @@ 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 = @@ -273,7 +271,7 @@ public void run() { Set allNodes = new HashSet(); while (!stopped.get() && !Thread.currentThread().isInterrupted()) { - XTraceContext.clearThreadContext(); + XTrace.stop(); try { event = eventQueue.take(); } catch (InterruptedException e) { @@ -361,10 +359,10 @@ class EventProcessor implements Runnable { @Override public void run() { - XTraceContext.clearThreadContext(); + XTrace.stop(); event.joinContext(); LOG.info("Processing the event " + event.toString()); - XTraceContext.logEvent(ContainerLauncherImpl.class, "ContainerLauncherImpl", "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. @@ -384,7 +382,7 @@ public void run() { break; } removeContainerIfDone(containerID); - XTraceContext.clearThreadContext(); + 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/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 c0fc462bedd0..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 @@ -21,7 +21,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; -import java.util.Collection; import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; @@ -55,8 +54,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; /** * Registers/unregisters to RM and sends heartbeats to RM. */ @@ -250,7 +248,7 @@ public void run() { } return; } - XTraceContext.clearThreadContext(); + 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 e7cb61787c29..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 @@ -19,7 +19,6 @@ package org.apache.hadoop.mapreduce.v2.app.rm; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -75,9 +74,9 @@ import org.apache.hadoop.yarn.util.RackResolver; import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; + +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** * Allocates the container from the ResourceManager scheduler. @@ -85,6 +84,7 @@ 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 @@ -191,7 +191,7 @@ public void run() { ContainerAllocatorEvent event; while (!stopped.get() && !Thread.currentThread().isInterrupted()) { - XTraceContext.clearThreadContext(); + XTrace.stop(); try { event = RMContainerAllocator.this.eventQueue.take(); } catch (InterruptedException e) { @@ -227,7 +227,7 @@ protected synchronized void heartbeat() throws Exception { scheduledRequests.assign(allocatedContainers); } - Collection start_context = XTraceContext.getThreadContext(); + Context start_context = XTrace.get(); int completedMaps = getJob().getAndJoinCompletedMaps(); int completedTasks = completedMaps + getJob().getAndJoinCompletedReduces(); @@ -248,7 +248,7 @@ protected synchronized void heartbeat() throws Exception { recalculateReduceSchedule = false; } - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); scheduleStats.updateAndLogIfChanged("After Scheduling: "); } @@ -297,7 +297,7 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) { event.joinContext(); recalculateReduceSchedule = true; if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) { - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "Processing CONTAINER_REQ Event"); + xtrace.log("Processing CONTAINER_REQ Event"); ContainerRequestEvent reqEvent = (ContainerRequestEvent) event; JobId jobId = getJob().getID(); int supportedMaxContainerCapability = @@ -320,7 +320,7 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) { } } //set the rounded off memory - XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", "Scheduling Map Container Request"); + xtrace.log("Scheduling Map Container Request"); reqEvent.rememberContext(); reqEvent.getCapability().setMemory(mapResourceReqt); scheduledRequests.addMap(reqEvent);//maps are immediately scheduled @@ -347,11 +347,11 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) { reqEvent.getCapability().setMemory(reduceResourceReqt); if (reqEvent.getEarlierAttemptFailed()) { //add to the front of queue for fail fast - XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", "Scheduling Fail-Fast Reduce Container Request"); + xtrace.log("Scheduling Fail-Fast Reduce Container Request"); reqEvent.rememberContext(); pendingReduces.addFirst(new ContainerRequest(reqEvent, PRIORITY_REDUCE)); } else { - XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", "Scheduling Reduce Container Request"); + 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 @@ -360,7 +360,7 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) { } else if ( event.getType() == ContainerAllocator.EventType.CONTAINER_DEALLOCATE) { - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "Processing CONTAINER_DEALLOCATE Event"); + xtrace.log("Processing CONTAINER_DEALLOCATE Event"); LOG.info("Processing the event " + event.toString()); @@ -382,7 +382,7 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) { } } else if ( event.getType() == ContainerAllocator.EventType.CONTAINER_FAILED) { - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "Processing CONTAINER_FAILED Event"); + xtrace.log("Processing CONTAINER_FAILED Event"); ContainerFailedEvent fEv = (ContainerFailedEvent) event; String host = getHost(fEv.getContMgrAddress()); containerFailedOnHost(host); @@ -535,31 +535,29 @@ public void scheduleReduces( @Private public void scheduleAllReduces() { - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator scheduleAllReduces", "Scheduling all reduces"); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("Scheduling all reduces"); + Context start_context = XTrace.get(); for (ContainerRequest req : pendingReduces) { req.joinContext(); - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator scheduleAllReduces", "Scheduling reduce"); + xtrace.log("Scheduling reduce"); req.rememberContext(); scheduledRequests.addReduce(req); - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); } pendingReduces.clear(); } @Private public void rampUpReduces(int rampUp) { - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator rampUpReduces", "Ramping up reduces", - "Ramp Up", rampUp); - //more reduce to be scheduled - Collection start_context = XTraceContext.getThreadContext(); + 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(); - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator rampUpReduces", "Request ramping up"); + xtrace.log("Request ramping up"); request.rememberContext(); scheduledRequests.addReduce(request); - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); } } @@ -910,7 +908,7 @@ else if (PRIORITY_REDUCE.equals(priority)) { continue; } - XTraceContext.clearThreadContext(); + XTrace.stop(); } assignContainers(allocatedContainers); @@ -923,17 +921,16 @@ else if (PRIORITY_REDUCE.equals(priority)) { LOG.info("Releasing unassigned and invalid container " + allocated + ". RM may have assignment issues"); containerNotAssigned(allocated); - XTraceContext.clearThreadContext(); + XTrace.stop(); } - XTraceContext.clearThreadContext(); + XTrace.stop(); } @SuppressWarnings("unchecked") private void containerAssigned(Container allocated, ContainerRequest assigned) { - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", - "Container Assigned", "Container ID", allocated.getId().toString(), "Assigned ID", assigned.attemptID); + xtrace.log("Container Assigned", "Container ID", allocated.getId(), "Assigned ID", assigned.attemptID); // Update resource requests decContainerReq(assigned); @@ -964,16 +961,14 @@ private ContainerRequest assignWithoutLocality(Container allocated) { LOG.info("Assigning container " + allocated + " to fast fail map"); assigned = assignToFailedMap(allocated); allocated.getId().joinContext(); - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "Assigned container for fast fail map", - "Container ID", allocated.getId(), "Assigned ID", assigned.attemptID); + 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(); - XTraceContext.logEvent(RMContainerAllocator.class, "RMContainerAllocator", "Assigned container to reduce", - "Container ID", allocated.getId(), "Assigned ID", assigned.attemptID); + xtrace.log("Assigned container to reduce", "Container ID", allocated.getId(), "Assigned ID", assigned.attemptID); } return assigned; @@ -982,7 +977,7 @@ private ContainerRequest assignWithoutLocality(Container allocated) { private void assignContainers(List allocatedContainers) { Iterator it = allocatedContainers.iterator(); while (it.hasNext()) { - XTraceContext.clearThreadContext(); + XTrace.stop(); Container allocated = it.next(); ContainerRequest assigned = assignWithoutLocality(allocated); if (assigned != null) { @@ -990,10 +985,10 @@ private void assignContainers(List allocatedContainers) { it.remove(); } } - XTraceContext.clearThreadContext(); + XTrace.stop(); assignMapsWithLocality(allocatedContainers); - XTraceContext.clearThreadContext(); + XTrace.stop(); } private ContainerRequest getContainerReqToReplace(Container allocated) { @@ -1072,7 +1067,7 @@ private void assignMapsWithLocality(List allocatedContainers) { Iterator it = allocatedContainers.iterator(); while(it.hasNext() && maps.size() > 0){ Container allocated = it.next(); - XTraceContext.clearThreadContext(); + XTrace.stop(); allocated.getId().joinContext(); Priority priority = allocated.getPriority(); assert PRIORITY_MAP.equals(priority); @@ -1089,8 +1084,7 @@ private void assignMapsWithLocality(List allocatedContainers) { if (maps.containsKey(tId)) { ContainerRequest assigned = maps.remove(tId); assigned.joinContext(); - XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", - "Assigned container based on host match", "Host", host); + xtrace.log("Assigned container based on host match", "Host", host); containerAssigned(allocated, assigned); it.remove(); JobCounterUpdateEvent jce = @@ -1105,12 +1099,12 @@ private void assignMapsWithLocality(List allocatedContainers) { } } } - XTraceContext.clearThreadContext(); + XTrace.stop(); // try to match all rack local it = allocatedContainers.iterator(); while(it.hasNext() && maps.size() > 0){ - XTraceContext.clearThreadContext(); + XTrace.stop(); Container allocated = it.next(); allocated.getId().joinContext(); Priority priority = allocated.getPriority(); @@ -1125,8 +1119,7 @@ private void assignMapsWithLocality(List allocatedContainers) { if (maps.containsKey(tId)) { ContainerRequest assigned = maps.remove(tId); assigned.joinContext(); - XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", - "Assigned container based on rack match", "Rack", rack); + xtrace.log("Assigned container based on rack match", "Rack", rack); containerAssigned(allocated, assigned); it.remove(); JobCounterUpdateEvent jce = @@ -1139,15 +1132,15 @@ private void assignMapsWithLocality(List allocatedContainers) { } break; } - XTraceContext.clearThreadContext(); + XTrace.stop(); } } - XTraceContext.clearThreadContext(); + XTrace.stop(); // assign remaining it = allocatedContainers.iterator(); while(it.hasNext() && maps.size() > 0){ - XTraceContext.clearThreadContext(); + XTrace.stop(); Container allocated = it.next(); allocated.getId().joinContext(); Priority priority = allocated.getPriority(); @@ -1155,7 +1148,7 @@ private void assignMapsWithLocality(List allocatedContainers) { TaskAttemptId tId = maps.keySet().iterator().next(); ContainerRequest assigned = maps.remove(tId); assigned.joinContext(); - XTraceContext.logEvent(RMContainerAllocator.class, "ContainerAllocator", "Assigned container based on * match"); + xtrace.log("Assigned container based on * match"); containerAssigned(allocated, assigned); it.remove(); JobCounterUpdateEvent jce = @@ -1165,9 +1158,9 @@ private void assignMapsWithLocality(List allocatedContainers) { if (LOG.isDebugEnabled()) { LOG.debug("Assigned based on * match"); } - XTraceContext.clearThreadContext(); + XTrace.stop(); } - XTraceContext.clearThreadContext(); + 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 cec9d3c930a6..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 @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -50,14 +49,14 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +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; @@ -167,20 +166,19 @@ protected void serviceInit(Configuration conf) throws Exception { protected AllocateResponse makeRemoteRequest() throws IOException { for (ResourceRequest r : ask) { - XTraceContext.clearThreadContext(); + XTrace.stop(); r.joinContext(); - XTraceContext.logEvent(RMContainerRequestor.class, "ContainerRequestor", "Requesting container from RM"); + xtrace.log("Requesting container from RM"); r.rememberContext(); } for (ContainerId i : release) { - XTraceContext.clearThreadContext(); + XTrace.stop(); i.joinContext(); - XTraceContext.logEvent(RMContainerRequestor.class, "ContainerRequestor", "Requesting RM release container", - "Container ID", i); + xtrace.log("Requesting RM release container", "Container ID", i); i.rememberContext(); } - XTraceContext.clearThreadContext(); + XTrace.stop(); AllocateRequest allocateRequest = @@ -209,20 +207,20 @@ protected AllocateResponse makeRemoteRequest() throws IOException { } for (Container x : allocateResponse.getAllocatedContainers()) { - XTraceContext.clearThreadContext(); + XTrace.stop(); x.getId().joinContext(); - XTraceContext.logEvent(RMContainerRequestor.class, "ContainerRequestor", "Container allocated by RM"); + xtrace.log("Container allocated by RM"); x.getId().rememberContext(); } for (ContainerStatus x : allocateResponse.getCompletedContainersStatuses()) { - XTraceContext.clearThreadContext(); + XTrace.stop(); x.getContainerId().joinContext(); - XTraceContext.logEvent(RMContainerRequestor.class, "ContainerRequestor", "RM acknowledged completed container"); + xtrace.log("RM acknowledged completed container"); x.getContainerId().rememberContext(); } - XTraceContext.clearThreadContext(); + XTrace.stop(); ask.clear(); release.clear(); 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 2f72d923ad6f..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 @@ -20,7 +20,6 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.util.Collection; import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -53,9 +52,6 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.util.Clock; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - public class DefaultSpeculator extends AbstractService implements Speculator { @@ -170,12 +166,10 @@ public DefaultSpeculator(Configuration conf, AppContext context, Clock clock) { @Override protected void serviceStart() throws Exception { - final Collection xtrace_context = XTraceContext.getThreadContext(); Runnable speculationBackgroundCore = new Runnable() { @Override public void run() { - XTraceContext.joinContext(xtrace_context); while (!stopped && !Thread.currentThread().isInterrupted()) { long backgroundRunStartTime = clock.getTime(); try { 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 ea880f923d6a..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 @@ -25,7 +25,6 @@ import java.io.IOException; import java.io.OutputStream; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -69,15 +68,15 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +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 = @@ -132,7 +131,7 @@ private class Job extends Thread implements TaskUmbilicalProtocol { boolean killed = false; private LocalDistributedCacheManager localDistributedCacheManager; - private Collection xtrace_context; + private final Context xtrace_context; public long getProtocolVersion(String protocol, long clientVersion) { return TaskUmbilicalProtocol.versionID; @@ -184,7 +183,7 @@ public Job(JobID jobid, String jobSubmitDir) throws IOException { jobs.put(id, this); - this.xtrace_context = XTraceContext.getThreadContext(); + this.xtrace_context = XTrace.get(); this.start(); } @@ -204,7 +203,7 @@ protected class MapTaskRunnable implements Runnable { private final Map mapOutputFiles; public volatile Throwable storedException; - private Collection xtrace_context; + private Context runnable_end_context; public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId, Map mapOutputFiles) { @@ -213,11 +212,9 @@ public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId, this.mapOutputFiles = mapOutputFiles; this.jobId = jobId; this.localConf = new JobConf(job); - this.xtrace_context = XTraceContext.getThreadContext(); } public void run() { - XTraceContext.setThreadContext(xtrace_context); try { TaskAttemptID mapId = new TaskAttemptID(new TaskID( jobId, TaskType.MAP, taskId), 0); @@ -250,8 +247,8 @@ public void run() { } catch (Throwable e) { this.storedException = e; } - this.xtrace_context = XTraceContext.getThreadContext(); - XTraceContext.clearThreadContext(); + this.runnable_end_context = XTrace.get(); + XTrace.stop(); } } @@ -354,7 +351,7 @@ protected ExecutorService createMapExecutor(int numMapTasks) { @Override public void run() { - XTraceContext.setThreadContext(xtrace_context); + XTrace.set(xtrace_context); JobID jobId = profile.getJobID(); JobContext jContext = new JobContextImpl(job, jobId); @@ -406,12 +403,12 @@ public void run() { throw ie; } - XTraceContext.clearThreadContext(); + XTrace.stop(); for (MapTaskRunnable r : taskRunnables) { - XTraceContext.joinContext(r.xtrace_context); + XTrace.join(r.runnable_end_context); } - XTraceContext.logEvent(Job.class, "Job", "Map tasks complete"); + 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 7e45f21e7e1b..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,10 +18,8 @@ package org.apache.hadoop.mapreduce.v2.api.records; -import java.util.Collection; - -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** *

    @@ -38,14 +36,14 @@ */ public abstract class TaskAttemptId implements Comparable { - private Collection xtrace_context; + private Context xtrace_context; public void rememberContext() { - this.xtrace_context = XTraceContext.getThreadContext(); + xtrace_context = XTrace.get(); } public void joinContext() { - XTraceContext.joinContext(this.xtrace_context); + XTrace.join(xtrace_context); } /** 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 66eed58a7132..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 @@ -19,18 +19,13 @@ package org.apache.hadoop.mapred; import java.io.IOException; -import java.util.Collection; import java.util.concurrent.LinkedBlockingQueue; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - class CleanupQueue { public static final Log LOG = @@ -109,12 +104,10 @@ private static class PathCleanupThread extends Thread { // cleanup queue which deletes files/directories of the paths queued up. private LinkedBlockingQueue queue = new LinkedBlockingQueue(); - private Collection xtrace_context; public PathCleanupThread() { setName("Directory/File cleanup thread"); setDaemon(true); - xtrace_context = XTraceContext.getThreadContext(); start(); } @@ -127,7 +120,6 @@ void addToQueue(PathDeletionContext[] contexts) { } public void run() { - XTraceContext.joinContext(xtrace_context); if (LOG.isDebugEnabled()) { LOG.debug(getName() + " started."); } 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 2028ec2f7823..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 @@ -17,14 +17,13 @@ */ package org.apache.hadoop.mapred; -import java.nio.ByteBuffer; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.TaskID; +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 @@ -32,7 +31,7 @@ public class IndexRecord { public long startOffset; public long rawLength; public long partLength; - public XTraceMetadata m; + public Context ctx; public IndexRecord() { } @@ -40,17 +39,18 @@ public IndexRecord(long startOffset, long rawLength, long partLength, long xtrac this.startOffset = startOffset; this.rawLength = rawLength; this.partLength = partLength; - if (xtrace_taskid!=0 && xtrace_opid!=0) { - byte[] taskid = ByteBuffer.allocate(8).putLong(xtrace_taskid).array(); - byte[] opid = ByteBuffer.allocate(8).putLong(xtrace_opid).array(); - m = new XTraceMetadata(new TaskID(taskid, 8), opid); + 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 (m!=null) { + if (ctx!=null) { try { - return ByteBuffer.wrap(m.getTaskId().get()).getLong(); + return XTraceMetadata.parseFrom(ctx.bytes()).getTaskID(); } catch (Exception e) { } } @@ -58,9 +58,11 @@ public long getXTraceTaskID() { } public long getXTraceOpID() { - if (m!=null) { + if (ctx!=null) { try { - return ByteBuffer.wrap(m.getOpId()).getLong(); + XTraceMetadata md = XTraceMetadata.parseFrom(ctx.bytes()); + if (md.getParentEventIDCount() > 0) + return md.getParentEventID(0); } catch (Exception e) { } } @@ -68,19 +70,19 @@ public long getXTraceOpID() { } public void rememberContext() { - m = XTraceContext.logMerge(); + ctx = XTrace.get(); } public void clearContext() { - m = null; + ctx = null; } public void joinContext() { - XTraceContext.joinContext(m); + XTrace.join(ctx); } public boolean hasContext() { - return m!=null; + return ctx!=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 92952763e067..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 @@ -29,9 +29,8 @@ import java.nio.IntBuffer; 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.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -76,9 +75,7 @@ import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.util.StringUtils; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +import edu.brown.cs.systems.xtrace.XTrace; /** A Map task. */ @InterfaceAudience.LimitedPrivate({"MapReduce"}) @@ -92,6 +89,7 @@ public class MapTask extends Task { 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; @@ -266,8 +264,7 @@ public synchronized boolean next(K key, V value) long nextRecIndex = skipIt.next(); long skip = 0; if (recIndex 0) { setPhase(TaskStatus.Phase.SORT); @@ -461,7 +458,7 @@ void runOldMapper(final JobConf job, closeQuietly(in); closeQuietly(collector); } - XTraceContext.logEvent(MapTask.class, "OldMapper", "Sort phase end"); + xtrace.log("OldMapper Sort phase end"); } /** @@ -748,7 +745,7 @@ void runNewMapper(final JobConf job, split = getSplitDetails(new Path(splitIndex.getSplitLocation()), splitIndex.getStartOffset()); LOG.info("Processing split: " + split); - XTraceContext.logEvent(MapTask.class, "NewMapper", "Processing split", "Split", split); + xtrace.log("NewMapper Processing split", "Split", split); org.apache.hadoop.mapreduce.RecordReader input = new NewTrackingRecordReader @@ -779,18 +776,18 @@ void runNewMapper(final JobConf job, try { input.initialize(split, mapperContext); - XTraceContext.logEvent(MapTask.class, "NewMapper", "Map start"); + xtrace.log("NewMapper Map start"); mapper.run(mapperContext); mapPhase.complete(); - XTraceContext.logEvent(MapTask.class, "NewMapper", "Map end"); - XTraceContext.logEvent(MapTask.class, "NewMapper", "Sort phase start"); + 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; - XTraceContext.logEvent(MapTask.class, "NewMapper", "Sort phase end"); + xtrace.log("NewMapper Sort phase end"); } finally { closeQuietly(input); closeQuietly(output, mapperContext); @@ -1110,7 +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 - XTraceContext.logEvent(MapTask.class, "MapTask startspill async", "Triggering asynchronous spill"); + xtrace.log("Triggering asynchronous spill"); startSpill(); final int avgRec = (int) (mapOutputByteCounter.getCounter() / @@ -1182,8 +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()); - XTraceContext.logEvent(MapOutputBuffer.class, "MapTask Output Buffer Collect", - "Record too large for in-memory buffer", "Message", e.getMessage()); + xtrace.log("Record too large for in-memory buffer", "Message", e.getMessage()); spillSingleRecord(key, value, partition); mapOutputRecordCounter.increment(1); return; @@ -1416,7 +1412,7 @@ public void write(byte b[], int off, int len) } // we have records we can spill; only spill if blocked if (kvindex != kvend) { - XTraceContext.logEvent(MapTask.class, "MapTask startspill sync", "Triggering synchronous spill"); + xtrace.log("Triggering synchronous spill"); startSpill(); // Blocked on this write, waiting for the spill just // initiated to finish. Instead of repositioning the marker @@ -1471,8 +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"); - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer flush", "Starting flush of map output"); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("Starting flush of map output"); spillLock.lock(); try { while (spillInProgress) { @@ -1530,24 +1525,24 @@ public void close() { } protected class SpillThread extends Thread { - private Collection xtrace_spillstart_context; - private Collection xtrace_spilldone_context; + 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 = XTraceContext.getThreadContext(); + xtrace_spillstart_context = XTrace.get(); } public void rememberSpillDoneContext() { - xtrace_spilldone_context = XTraceContext.getThreadContext(); + xtrace_spilldone_context = XTrace.get(); } public void joinSpillDoneContext() { - XTraceContext.joinContext(xtrace_spilldone_context); + XTrace.join(xtrace_spilldone_context); xtrace_spilldone_context = null; } public void joinSpillStartContext() { - XTraceContext.joinContext(xtrace_spillstart_context); + XTrace.join(xtrace_spillstart_context); xtrace_spillstart_context = null; } @@ -1557,13 +1552,13 @@ public void run() { spillThreadRunning = true; try { while (true) { - XTraceContext.clearThreadContext(); + XTrace.stop(); spillDone.signal(); while (!spillInProgress) { spillReady.await(); } joinSpillStartContext(); - XTraceContext.logEvent(SpillThread.class, "SpillThread", "Spill Thread notified"); + xtrace.log("Spill Thread notified"); try { spillLock.unlock(); sortAndSpill(); @@ -1622,7 +1617,7 @@ private void startSpill() { 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 - XTraceContext.logEvent(MapTask.class, "MapOutputBuffer sortAndSpill", "Beginning spill", "Spill Number", numSpills); + 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 @@ -1643,20 +1638,19 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, (kvstart >= kvend ? kvstart : kvmeta.capacity() + kvstart) / NMETA; - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer sort", "Sorting buffer contents"); + 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(); - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", - "Spilling partitions to disk", "NumPartitions", partitions, "File", filename); - Collection start_context = XTraceContext.getThreadContext(); - Collection end_context = new XTraceMetadataCollection(); + 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) { - XTraceContext.setThreadContext(start_context); - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", "Spilling partition", "Partition Number", i); + XTrace.set(start_context); + xtrace.log("Spilling partition", "Partition Number", i); IFile.Writer writer = null; try { long segmentStart = out.getPos(); @@ -1678,11 +1672,10 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, ++spillcount; } if (spillcount > 0) { - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", - "Spilled records directly without combine", "Partition", i, "SpillCount", spillcount); + xtrace.log("Spilled records directly without combine", "Partition", i, "SpillCount", spillcount); rec.rememberContext(); } else { - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", "No records to spill", "Partition", i); + xtrace.log("No records to spill", "Partition", i); rec.clearContext(); } } else { @@ -1699,11 +1692,10 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, RawKeyValueIterator kvIter = new MRResultIterator(spstart, spindex); combinerRunner.combine(kvIter, combineCollector); - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", - "Spilled records with combine", "Partition", i, "SpillCount", (spindex-spstart)); + xtrace.log("Spilled records with combine", "Partition", i, "SpillCount", (spindex-spstart)); rec.rememberContext(); } else { - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spill", "No records to spill", "Partition", i); + xtrace.log("No records to spill", "Partition", i); rec.clearContext(); } } @@ -1720,10 +1712,11 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, writer = null; } finally { if (null != writer) writer.close(); - end_context = XTraceContext.getThreadContext(end_context); + end_contexts.add(XTrace.get()); } } - XTraceContext.joinContext(end_context); + for (edu.brown.cs.systems.xtrace.Context ctx : end_contexts) + XTrace.join(ctx); if (totalIndexCacheMemory >= indexCacheMemoryLimit) { // create spill index file @@ -1737,7 +1730,7 @@ private void sortAndSpill() throws IOException, ClassNotFoundException, spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH; } LOG.info("Finished spill " + numSpills); - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer sortAndSpill", "Finished spill", "Spill Number", numSpills); + xtrace.log("Finished spill", "Spill Number", numSpills); ++numSpills; } finally { if (out != null) out.close(); @@ -1776,8 +1769,7 @@ 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); - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer spillSingleRecord", - "Spilled single record", "Partition", partition); + xtrace.log("Spilled single record", "Partition", partition); rec.rememberContext(); } writer.close(); @@ -1903,7 +1895,7 @@ private void mergeParts() throws IOException, InterruptedException, } if (numSpills == 1) { //the spill is the final output - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer mergeParts", "Single spill, merge unnecessary"); + xtrace.log("Single spill, merge unnecessary"); sameVolRename(filename[0], mapOutputFile.getOutputFileForWriteInVolume(filename[0])); @@ -1912,15 +1904,15 @@ private void mergeParts() throws IOException, InterruptedException, mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0])); } else { SpillRecord spillRec = indexCacheList.get(0); - Collection start_context = XTraceContext.getThreadContext(); + 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(); - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer rename", "Spill file containing partition was renamed"); + xtrace.log("Spill file containing partition was renamed"); rec.rememberContext(); spillRec.putIndex(rec, i); - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); } } spillRec.writeToFile( @@ -1970,10 +1962,10 @@ private void mergeParts() throws IOException, InterruptedException, IndexRecord rec = new IndexRecord(); final SpillRecord spillRec = new SpillRecord(partitions); - Collection start_context = XTraceContext.getThreadContext(); - Collection end_contexts = new XTraceMetadataCollection(); + edu.brown.cs.systems.xtrace.Context start_context = XTrace.get(); + Collection end_contexts = new HashSet(); for (int parts = 0; parts < partitions; parts++) { - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); //create the segments to be merged List> segmentList = new ArrayList>(numSpills); @@ -1997,8 +1989,7 @@ private void mergeParts() throws IOException, InterruptedException, // sort the segments only if there are intermediate merges boolean sortSegments = segmentList.size() > mergeFactor; //merge - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer mergeParts", "Merging parts from multiple spills", - "Partition", parts, "Num Spills", numSpills); + xtrace.log("Merging parts from multiple spills", "Partition", parts, "Num Spills", numSpills); @SuppressWarnings("unchecked") RawKeyValueIterator kvIter = Merger.merge(job, rfs, keyClass, valClass, codec, @@ -2013,7 +2004,7 @@ private void mergeParts() throws IOException, InterruptedException, new Writer(job, finalOut, keyClass, valClass, codec, spilledRecordsCounter); if (combinerRunner == null || numSpills < minSpillsForCombine) { - XTraceContext.logEvent(MapOutputBuffer.class, "MapOutputBuffer skipCombine", "Skipping combine", + xtrace.log("Skipping combine", "CombinerRunnerIsNull", combinerRunner==null, "MinSpillsForCombine", minSpillsForCombine, "Partition", parts, "Num Spills", numSpills); Merger.writeFile(kvIter, writer, reporter, job); @@ -2034,16 +2025,16 @@ private void mergeParts() throws IOException, InterruptedException, rec.rememberContext(); spillRec.putIndex(rec, parts); - end_contexts = XTraceContext.getThreadContext(end_contexts); + end_contexts.add(XTrace.get()); } - XTraceContext.setThreadContext(end_contexts); + 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); } - XTraceContext.logEvent(MapTask.class, "MapTask", "Final output written", - "Final Index File", finalIndexFile.toString(), "Final Output File", finalOutputFile.toString()); + 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 87f9202dec62..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,7 +43,7 @@ import org.apache.hadoop.util.Progress; import org.apache.hadoop.util.Progressable; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** * Merger is an utility class used by the Map and Reduce tasks for merging @@ -52,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 @@ -568,8 +569,7 @@ RawKeyValueIterator merge(Class keyClass, Class valueClass, Progress mergePhase) throws IOException { LOG.info("Merging " + segments.size() + " sorted segments"); - XTraceContext.logEvent(Merger.class, "Merger merge", "Merging sorted segments", - "Num Segments", + segments.size()); + 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 fa150e29d749..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,7 +56,7 @@ import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.ReflectionUtils; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** A Reduce task. */ @InterfaceAudience.Private @@ -71,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; @@ -314,7 +315,7 @@ private void writeSkippedRec(KEY key, VALUE value) throws IOException{ public void run(JobConf job, final TaskUmbilicalProtocol umbilical) throws IOException, InterruptedException, ClassNotFoundException { - XTraceContext.logEvent(ReduceTask.class, "ReduceTask", "ReduceTask running"); + xtrace.log("ReduceTask running"); job.setBoolean(JobContext.SKIP_RECORDS, isSkipping()); @@ -646,12 +647,12 @@ public boolean next() throws IOException { committer, reporter, comparator, keyClass, valueClass); - XTraceContext.logEvent(ReduceTask.class, "NewReducer", "Running reduce start"); + xtrace.log("Running reduce start"); try { reducer.run(reducerContext); } finally { trackedRW.close(reducerContext); - XTraceContext.logEvent(ReduceTask.class, "NewReducer", "Running reduce end"); + 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 99b5b08c9b6e..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 @@ -38,9 +38,6 @@ import org.apache.hadoop.io.SecureIOUtils; import org.apache.hadoop.util.PureJavaCrc32; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - @InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceStability.Unstable public class SpillRecord { 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 4de47aebdf39..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 @@ -27,6 +27,7 @@ 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; @@ -42,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; @@ -55,24 +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.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** * Base class for tasks. @@ -80,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; @@ -147,12 +147,13 @@ static synchronized String getOutputName(int partition) { //////////////////////////////////////////// // X-Trace Methods //////////////////////////////////////////// - private Collection xtrace_context = new XTraceMetadataCollection(); + private Collection xtrace_contexts = new HashSet(); public void rememberContext() { - XTraceContext.getThreadContext(xtrace_context); + xtrace_contexts.add(XTrace.get()); } public void joinContext() { - XTraceContext.joinContext(xtrace_context); + for (Context ctx : xtrace_contexts) + XTrace.join(ctx); } @@ -338,7 +339,7 @@ protected void setWriteSkipRecs(boolean writeSkipRecs) { protected void reportFatalError(TaskAttemptID id, Throwable throwable, String logMsg) { LOG.fatal(logMsg); - XTraceContext.logEvent(Task.class, "FatalError", "Fatal error occurred", "Message", logMsg); + xtrace.log(logMsg); Throwable tCause = throwable.getCause(); String cause = tCause == null ? StringUtils.stringifyException(throwable) @@ -347,8 +348,9 @@ protected void reportFatalError(TaskAttemptID id, Throwable throwable, umbilical.fatalError(id, cause); } catch (IOException ioe) { LOG.fatal("Failed to contact the tasktracker", ioe); - XTraceContext.logEvent(Task.class, "FatalError", "Failed to contact the tasktracker", "Exit Code", -1); - XTraceContext.joinParentProcess(); + xtrace.log("Failed to contact the tasktracker", "Exit Code", -1); + // TODO: XTrace parent process stuff +// XTraceContext.joinParentProcess(); System.exit(-1); } } @@ -637,13 +639,11 @@ public class TaskReporter * Using AtomicBoolean since we need an atomic read & reset method. */ private AtomicBoolean progressFlag = new AtomicBoolean(false); - private Collection xtrace; TaskReporter(Progress taskProgress, TaskUmbilicalProtocol umbilical) { this.umbilical = umbilical; this.taskProgress = taskProgress; - this.xtrace = XTraceContext.getThreadContext(); } // getters and setters for flag @@ -723,7 +723,6 @@ public InputSplit getInputSplit() throws UnsupportedOperationException { * let the parent know that it's alive. It also pings the parent to see if it's alive. */ public void run() { - XTraceContext.setThreadContext(xtrace); final int MAX_RETRIES = 3; int remainingRetries = MAX_RETRIES; // get current flag value and reset it as well @@ -763,9 +762,10 @@ public void run() { // came back up), kill ourselves if (!taskFound) { LOG.warn("Parent died. Exiting "+taskId); - XTraceContext.logEvent(Task.class, "Task", "Parent died, exiting", "Exit Code", 66); + xtrace.log("Parent died, exiting", "Exit Code", 66); resetDoneFlag(); - XTraceContext.joinParentProcess(); +// // TODO: XTrace join parent process +// XTraceContext.joinParentProcess(); System.exit(66); } @@ -775,14 +775,14 @@ public void run() { catch (Throwable t) { LOG.info("Communication exception: " + StringUtils.stringifyException(t)); remainingRetries -=1; - XTraceContext.logEvent(Task.class, "Task", "Communication exception "+t.getClass().getName(), - "Message", t.getMessage(), "Retries Remaining", remainingRetries); + 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); - XTraceContext.logEvent(Task.class, "Task", "No retries remaining, killing task", "Exit Code", 65); + xtrace.log("No retries remaining, killing task", "Exit Code", 65); resetDoneFlag(); - XTraceContext.joinParentProcess(); +// // TODO: XTrace join parent process +// XTraceContext.joinParentProcess(); System.exit(65); } } @@ -1024,7 +1024,7 @@ public void done(TaskUmbilicalProtocol umbilical, LOG.info("Task:" + taskId + " is done." + " And is in the process of committing"); - XTraceContext.logEvent(Task.class, "Task committing", "Task is done and in the process of committing"); + xtrace.log("Task is done and in the process of committing"); updateCounters(); boolean commitRequired = isCommitRequired(); @@ -1041,11 +1041,11 @@ public void done(TaskUmbilicalProtocol umbilical, } catch (IOException ie) { LOG.warn("Failure sending commit pending: " + StringUtils.stringifyException(ie)); - XTraceContext.logEvent(Task.class, "Task committing fail", "Failure sending commit pending: "+ie.getClass().getName(), - "Message", ie.getMessage(), "Retries Remaining", retries); + xtrace.log("Failure sending commit pending: "+ie.getClass().getName(), "Message", ie.getMessage(), "Retries Remaining", retries); if (--retries == 0) { - XTraceContext.logEvent(Task.class, "Task exiting", "No retries remaining for task commit, killing task", "Exit Code", 67); - XTraceContext.joinParentProcess(); + xtrace.log("No retries remaining for task commit, killing task", "Exit Code", 67); +// // TODO: Xtrace join parent process +// XTraceContext.joinParentProcess(); System.exit(67); } } @@ -1091,8 +1091,9 @@ public void statusUpdate(TaskUmbilicalProtocol umbilical) try { if (!umbilical.statusUpdate(getTaskID(), taskStatus)) { LOG.warn("Parent died. Exiting "+taskId); - XTraceContext.logEvent(Task.class, "Task exiting", "Parent died, exiting", "Exit Code", 66); - XTraceContext.joinParentProcess(); + xtrace.log("Parent died, exiting", "Exit Code", 66); +// // TODO: XTrace join parent process +// XTraceContext.joinParentProcess(); System.exit(66); } taskStatus.clearStatus(); @@ -1148,16 +1149,14 @@ private void sendDone(TaskUmbilicalProtocol umbilical) throws IOException { int retries = MAX_RETRIES; while (true) { try { - XTraceContext.logEvent(Task.class, "Task done", "Notifying AppMaster that task is done"); + 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)); - XTraceContext.logEvent(Task.class, "Task done signalling failure", - "Failure signalling completion: "+ie.getClass().getName(), "Message", ie.getMessage(), - "Retries Remaining", retries); + xtrace.log("Failure signalling completion: "+ie.getClass().getName(), "Message", ie.getMessage(), "Retries Remaining", retries); if (--retries == 0) { throw ie; } @@ -1170,7 +1169,7 @@ private void commit(TaskUmbilicalProtocol umbilical, org.apache.hadoop.mapreduce.OutputCommitter committer ) throws IOException { int retries = MAX_RETRIES; - XTraceContext.logEvent(Task.class, "Await commit approval", "Awaiting commit approval from AM"); + xtrace.log("Awaiting commit approval from AM"); while (true) { try { while (!umbilical.canCommit(taskId)) { @@ -1185,13 +1184,14 @@ private void commit(TaskUmbilicalProtocol umbilical, } catch (IOException ie) { LOG.warn("Failure asking whether task can commit: " + StringUtils.stringifyException(ie)); - XTraceContext.logEvent(Task.class, "Commit approval failure", "Failure asking whether task can commit: "+ie.getClass().getName(), + 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); - XTraceContext.logEvent(Task.class, "Commit approval exiting", "Maximum retries reached, discarding output and exiting", "Exit Code", 68); - XTraceContext.joinParentProcess(); + xtrace.log("Maximum retries reached, discarding output and exiting", "Exit Code", 68); +// // TODO: XTrace join parent process +// XTraceContext.joinParentProcess(); System.exit(68); } } @@ -1200,14 +1200,13 @@ private void commit(TaskUmbilicalProtocol umbilical, // task can Commit now try { LOG.info("Task " + taskId + " is allowed to commit now"); - XTraceContext.logEvent(Task.class, "Commit approved", "Task is allowed to commit, committing"); + xtrace.log("Task is allowed to commit, committing"); committer.commitTask(taskContext); return; } catch (IOException iee) { LOG.warn("Failure committing: " + StringUtils.stringifyException(iee)); - XTraceContext.logEvent(Task.class, "Commit failed", "Failure committing: "+iee.getClass().getName()+", discarding output", - "Message", iee.getMessage()); + 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; @@ -1238,7 +1237,7 @@ void taskCleanup(TaskUmbilicalProtocol umbilical) getProgress().setStatus("cleanup"); statusUpdate(umbilical); LOG.info("Runnning cleanup for the task"); - XTraceContext.logEvent(Task.class, "TaskCleanup", "Runnning cleanup for the task"); + xtrace.log("Runnning cleanup for the task"); // do the cleanup committer.abortTask(taskContext); } @@ -1252,11 +1251,11 @@ protected void runJobCleanupTask(TaskUmbilicalProtocol umbilical, statusUpdate(umbilical); // do the cleanup LOG.info("Cleaning up job"); - XTraceContext.logEvent(Task.class, "JobCleanupTask", "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()); - XTraceContext.logEvent(Task.class, "JobCleanupTask", "Aborting job with runstate : " + jobRunStateForCleanup.name()); + xtrace.log("Aborting job with runstate : " + jobRunStateForCleanup.name()); if (conf.getUseNewMapper()) { committer.abortJob(jobContext, jobRunStateForCleanup); } else { @@ -1265,11 +1264,11 @@ protected void runJobCleanupTask(TaskUmbilicalProtocol umbilical, oldCommitter.abortJob(jobContext, jobRunStateForCleanup); } } else if (jobRunStateForCleanup == JobStatus.State.SUCCEEDED){ - XTraceContext.logEvent(Task.class, "JobCleanupTask", "Committing job"); + xtrace.log("Committing job"); LOG.info("Committing job"); committer.commitJob(jobContext); } else { - XTraceContext.logEvent(Task.class, "JobCleanupTask", "Invalid state of the job for cleanup. Found state " + xtrace.log("Invalid state of the job for cleanup. Found state " + jobRunStateForCleanup + ", but was expecting " + JobStatus.State.SUCCEEDED + ", " + JobStatus.State.FAILED + " or " @@ -1699,10 +1698,10 @@ public void combine(RawKeyValueIterator iterator, committer, reporter, comparator, keyClass, valueClass); - XTraceContext.logEvent(NewCombinerRunner.class, "Combiner start", "Combining map outputs", + xtrace.log("Combining map outputs", "Combiner", reducer.getClass().getName(), "KeyClass", keyClass, "ValClass", valueClass); reducer.run(reducerContext); - XTraceContext.logEvent(NewCombinerRunner.class, "Combiner end", "Combine complete"); + 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 f49f91462b38..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,26 +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 edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - -import java.io.IOException; -import java.util.Collection; -import java.util.concurrent.*; +import org.apache.hadoop.util.ReflectionUtils; /** * Multithreaded implementation for @link org.apache.hadoop.mapred.MapRunnable. @@ -209,7 +208,6 @@ private class MapperInvokeRunable implements Runnable { private V1 value; private OutputCollector output; private Reporter reporter; - private Collection xtrace_context; /** * Collecting all required parameters to execute a Mapper.map call. @@ -227,7 +225,6 @@ public MapperInvokeRunable(K1 key, V1 value, this.value = value; this.output = output; this.reporter = reporter; - this.xtrace_context = XTraceContext.getThreadContext(); } /** @@ -237,7 +234,6 @@ public MapperInvokeRunable(K1 key, V1 value, * */ public void run() { - XTraceContext.setThreadContext(xtrace_context); try { // map pair to output MultithreadedMapRunner.this.mapper.map(key, value, output, reporter); @@ -264,7 +260,6 @@ public void run() { } } } - XTraceContext.clearThreadContext(); } } 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 b58cb1b71fe0..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 @@ -42,7 +42,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; -import edu.berkeley.xtrace.XTraceContext; /** * Provides a way to access information about the map/reduce cluster. 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 4a2a4b6d5ba4..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 @@ -22,13 +22,12 @@ import java.io.StringWriter; import java.net.URI; import java.security.PrivilegedExceptionAction; -import java.util.Collection; 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; @@ -41,8 +40,8 @@ import org.apache.hadoop.mapreduce.util.ConfigUtil; import org.apache.hadoop.util.StringUtils; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** * The job submitter's view of the Job. @@ -80,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 @@ -115,7 +115,7 @@ public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL } private JobState state = JobState.DEFINE; private JobStatus status; - private Collection status_xtrace; + private Context status_xtrace; private long statustime; private Cluster cluster; @@ -145,9 +145,9 @@ public Job(Configuration conf, String jobName) throws IOException { StringWriter full = new StringWriter(); Configuration.dumpConfiguration(conf, full); if (!"".equals(conf.getJobName())) { - XTraceContext.startTrace("Hadoop Job", "Initializing Job", conf.getJobName()); + xtrace.log("Initializing Job", conf.getJobName()); } else { - XTraceContext.startTrace("Hadoop Job", "Initializing Job"); + xtrace.log("Initializing Job"); } } @@ -322,8 +322,8 @@ synchronized void ensureFreshStatus() * @throws IOException */ synchronized void updateStatus() throws IOException { - Collection start_context = XTraceContext.getThreadContext(); - XTraceContext.clearThreadContext(); + Context start_context = XTrace.get(); + XTrace.stop(); try { this.status = ugi.doAs(new PrivilegedExceptionAction() { @Override @@ -335,8 +335,8 @@ public JobStatus run() throws IOException, InterruptedException { catch (InterruptedException ie) { throw new IOException(ie); } finally { - this.status_xtrace = XTraceContext.getThreadContext(); - XTraceContext.setThreadContext(start_context); + this.status_xtrace = XTrace.get(); + XTrace.set(start_context); } if (this.status == null) { throw new IOException("Job status not available "); @@ -351,7 +351,7 @@ public JobStatus getStatus() throws IOException, InterruptedException { } public void joinStatusXTraceContext() { - XTraceContext.joinContext(this.status_xtrace); + XTrace.join(this.status_xtrace); } private void setStatus(JobStatus status) { @@ -1306,12 +1306,12 @@ public JobStatus run() throws IOException, InterruptedException, public boolean waitForCompletion(boolean verbose ) throws IOException, InterruptedException, ClassNotFoundException { - - XTraceContext.startTrace("MapReduce Job", "Preparing Job"); + + xtrace.log("Preparing Job"); if (state == JobState.DEFINE) { submit(); } - XTraceContext.logEvent(Job.class, "MapReduce Job", "Submitted Job", "Job ID", getJobID()); + xtrace.log("Submitted Job", "Job ID", getJobID()); if (verbose) { monitorAndPrintJob(); } else { @@ -1329,9 +1329,9 @@ 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()) { - XTraceContext.logEvent(Job.class, "MapReduce Job", "Job finished successfully", "Tag", maps+" Maps", "Tag", reds+" Reduces"); + xtrace.log("Job finished successfully", "Tag", maps+" Maps", "Tag", reds+" Reduces"); } else { - XTraceContext.logEvent(Job.class, "MapReduce Job", "Job failed", "Tag", maps+" Maps", "Tag", reds+" Reduces"); + xtrace.log("Job failed", "Tag", maps+" Maps", "Tag", reds+" Reduces"); } return isSuccessful(); @@ -1391,11 +1391,11 @@ public boolean monitorAndPrintJob() boolean success = isSuccessful(); if (success) { LOG.info("Job " + jobId + " completed successfully"); - XTraceContext.logEvent(Job.class, "Job", "Job completed successfully"); + xtrace.log("Job completed successfully"); } else { LOG.info("Job " + jobId + " failed with state " + status.getState() + " due to: " + status.getFailureInfo()); - XTraceContext.logEvent(Job.class, "Job", "Job failed with state "+status.getState()); + 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 001574e31cbc..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; -import edu.berkeley.xtrace.XTraceContext; /** * 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 2f2c7f80bd5d..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; @@ -25,7 +27,6 @@ import java.net.UnknownHostException; import java.security.NoSuchAlgorithmException; import java.util.Arrays; -import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -46,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,13 +62,14 @@ import org.codehaus.jackson.map.ObjectMapper; import com.google.common.base.Charsets; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceProcess; + +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; @@ -340,8 +340,8 @@ private void copyAndConfigureFiles(Job job, Path jobSubmitDir) JobStatus submitJobInternal(Job job, Cluster cluster) throws ClassNotFoundException, InterruptedException, IOException { - XTraceContext.logEvent(JobSubmitter.class, "JobSubmitter", "Submitting Job"); - Collection start_context = XTraceContext.getThreadContext(); + xtrace.log("Submitting Job"); + Context start_context = XTrace.get(); //validate the jobs output specs checkSpecs(job); @@ -435,8 +435,8 @@ JobStatus submitJobInternal(Job job, Cluster cluster) } - XTraceContext.joinContext(start_context); - XTraceContext.logEvent(JobSubmitter.class, "JobSubmitter", "Job submission complete"); + 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/lib/chain/Chain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java index 16bace9c4052..384524a92ddf 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import java.util.Map; @@ -42,9 +41,6 @@ import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer; import org.apache.hadoop.util.ReflectionUtils; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - /** * The Chain class provides all the common functionality for the * {@link ChainMapper} and the {@link ChainReducer} classes. @@ -305,7 +301,6 @@ private class MapRunner extends Thread { private Mapper.Context chainContext; private RecordReader rr; private RecordWriter rw; - private Collection xtrace_context; public MapRunner(Mapper mapper, Mapper.Context mapperContext, @@ -315,12 +310,10 @@ public MapRunner(Mapper mapper, this.rr = rr; this.rw = rw; this.chainContext = mapperContext; - this.xtrace_context = XTraceContext.getThreadContext(); } @Override public void run() { - XTraceContext.joinContext(xtrace_context); if (getThrowable() != null) { return; } @@ -340,7 +333,6 @@ private class ReduceRunner extends Thread { private Reducer reducer; private Reducer.Context chainContext; private RecordWriter rw; - private Collection xtrace_context; ReduceRunner(Reducer.Context context, Reducer reducer, @@ -349,12 +341,10 @@ private class ReduceRunner extends Thread { this.reducer = reducer; this.chainContext = context; this.rw = rw; - this.xtrace_context = XTraceContext.getThreadContext(); } @Override public void run() { - XTraceContext.joinContext(xtrace_context); try { reducer.run(chainContext); rw.close(chainContext); 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 ece83735d8c6..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,16 +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 edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; +import org.apache.hadoop.util.ReflectionUtils; /** * Multithreaded implementation for @link org.apache.hadoop.mapreduce.Mapper. @@ -255,7 +251,6 @@ private class MapRunner extends Thread { private Context subcontext; private Throwable throwable; private RecordReader reader = new SubMapRecordReader(); - private Collection xtrace_context; MapRunner(Context context) throws IOException, InterruptedException { mapper = ReflectionUtils.newInstance(mapClass, @@ -270,12 +265,10 @@ private class MapRunner extends Thread { outer.getInputSplit()); subcontext = new WrappedMapper().getMapContext(mapContext); reader.initialize(context.getInputSplit(), context); - this.xtrace_context = XTraceContext.getThreadContext(); } @Override public void run() { - XTraceContext.joinContext(xtrace_context); try { mapper.run(subcontext); reader.close(); 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 7c9a0d622186..f5d2d856330f 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 @@ -48,12 +48,12 @@ import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +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 */ @@ -92,8 +92,8 @@ private static enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP, protected HttpURLConnection connection; private volatile boolean stopped = false; - private Collection initial_xtrace_context; - private Collection copy_contexts; + private final Context xtrace_start_context; + private final Collection copy_contexts = new HashSet(); private static boolean sslShuffle; private static SSLFactory sslFactory; @@ -156,7 +156,7 @@ public Fetcher(JobConf job, TaskAttemptID reduceId, } } - initial_xtrace_context = XTraceContext.getThreadContext(); + xtrace_start_context = XTrace.get(); } public void run() { @@ -167,7 +167,7 @@ public void run() { // If merge is on, block merger.waitForResource(); - XTraceContext.setThreadContext(initial_xtrace_context); + XTrace.set(xtrace_start_context); // Get a host to shuffle from host = scheduler.getHost(); @@ -180,8 +180,8 @@ public void run() { scheduler.freeHost(host); metrics.threadFree(); } - copy_contexts = XTraceContext.getThreadContext(copy_contexts); - XTraceContext.clearThreadContext(); + copy_contexts.add(XTrace.get()); + XTrace.stop(); } } } catch (InterruptedException ie) { @@ -214,7 +214,8 @@ public void shutDown() throws InterruptedException { } public void joinContexts() { - XTraceContext.joinContext(copy_contexts); + for (Context ctx : copy_contexts) + XTrace.join(ctx); } @VisibleForTesting @@ -269,8 +270,7 @@ protected void copyFromHost(MapHost host) throws IOException { LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: " + maps); } - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Fetching map outputs from mapper", - "Num Maps", maps.size(), "Host", host, "TaskAttemptIDs", 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); @@ -279,7 +279,7 @@ protected void copyFromHost(MapHost host) throws IOException { DataInputStream input = null; try { URL url = getMapOutputURL(host, maps); - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Connecting to map output on host", "URL", url); + xtrace.log("Connecting to map output on host", "URL", url); openConnection(url); if (stopped) { abortConnect(host, remaining); @@ -294,7 +294,9 @@ protected void copyFromHost(MapHost host) throws IOException { // put url hash into http header connection.addRequestProperty( SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash); - connection.addRequestProperty("X-Trace", XTraceContext.logMerge().toString()); + String xtrace_string = XTrace.base64(); + if (xtrace_string!=null) + connection.addRequestProperty("X-Trace", xtrace_string); // set the read timeout connection.setReadTimeout(readTimeout); // put shuffle version into http header @@ -315,10 +317,9 @@ protected void copyFromHost(MapHost host) throws IOException { if (rc != HttpURLConnection.HTTP_OK) { String xtrace_context = connection.getHeaderField("X-Trace"); if (xtrace_context!=null) { - XTraceContext.joinContext(XTraceMetadata.createFromString(xtrace_context)); + XTrace.join(Context.parse(xtrace_context)); } - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Got invalid response code " + rc + " from host", - "URL", url, "Message", connection.getResponseMessage()); + 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()); @@ -344,8 +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); - XTraceContext.logEvent(Fetcher.class, "Fetcher","Failed to connect to host: "+ie.getClass().getName(), - "Host", host, "Remaining Outputs", remaining.size(), "Message", ie.getMessage()); + 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 @@ -367,20 +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; - Collection start_context = XTraceContext.getThreadContext(); - Collection end_contexts = new XTraceMetadataCollection(); + Context start_context = XTrace.get(); + Collection end_contexts = new HashSet(); int initialSize = remaining.size(); while (!remaining.isEmpty() && failedTasks == null) { - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); failedTasks = copyMapOutput(host, input, remaining); - end_contexts = XTraceContext.getThreadContext(end_contexts); + end_contexts.add(XTrace.get()); } - XTraceContext.joinContext(end_contexts); + for (Context ctx : end_contexts) + XTrace.join(ctx); if(failedTasks != null && failedTasks.length > 0) { LOG.warn("copyMapOutput failed for tasks "+Arrays.toString(failedTasks)); - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Failed to copy map output for some tasks", - "Failed 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); } @@ -388,8 +388,7 @@ protected void copyFromHost(MapHost host) throws IOException { // Sanity check if (failedTasks == null && !remaining.isEmpty()) { - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Server didn't return all expected map outputs", - "Remaining", remaining.size()); + 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."); } @@ -397,8 +396,7 @@ protected void copyFromHost(MapHost host) throws IOException { int failed = remaining.size(); int copied = initialSize - failed; - XTraceContext.logEvent(Fetcher.class, "Fetching complete", "Fetching complete", - "Num Succeeded", copied, "Num Failed", failed); + xtrace.log("Fetching complete", "Num Succeeded", copied, "Num Failed", failed); input.close(); input = null; @@ -430,7 +428,7 @@ private TaskAttemptID[] copyMapOutput(MapHost host, try { ShuffleHeader header = new ShuffleHeader(); header.readFields(input); - XTraceContext.clearThreadContext(); + XTrace.stop(); header.joinContext(); mapId = TaskAttemptID.forName(header.mapId); compressedLength = header.compressedLength; @@ -439,8 +437,7 @@ private TaskAttemptID[] copyMapOutput(MapHost host, } catch (IllegalArgumentException e) { badIdErrs.increment(1); LOG.warn("Invalid map id ", e); - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Invalid map ID: "+e.getClass().getName(), - "Message", e.toString(), "Map ID", mapId); + 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()]); } @@ -458,13 +455,12 @@ private TaskAttemptID[] copyMapOutput(MapHost host, } // Get the location for the map output - either in-memory or on-disk - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Reserving location for map output"); + xtrace.log("Reserving location for map output"); mapOutput = merger.reserve(mapId, decompressedLength, id); // Check if we can shuffle *now* ... if (mapOutput == null) { - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Merge Manager instructed fetcher to wait", - "Fetcher ID", id); + xtrace.log("Merge Manager instructed fetcher to wait", "Fetcher ID", id); //Not an error but wait to process data. return EMPTY_ATTEMPT_ID_ARRAY; } @@ -477,7 +473,7 @@ private TaskAttemptID[] copyMapOutput(MapHost host, LOG.info("fetcher#" + id + " about to shuffle output of map " + mapOutput.getMapId() + " decomp: " + decompressedLength + " len: " + compressedLength + " to " + mapOutput.getDescription()); - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Shuffling ouputs from mapper", + 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, @@ -501,7 +497,7 @@ private TaskAttemptID[] copyMapOutput(MapHost host, LOG.info("fetcher#" + id + " failed to read map header" + mapId + " decomp: " + decompressedLength + ", " + compressedLength, ioe); - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Fetcher failed to read map header: "+ioe.getClass().getName(), + 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) { @@ -513,7 +509,7 @@ private TaskAttemptID[] copyMapOutput(MapHost host, LOG.warn("Failed to shuffle output of " + mapId + " from " + host.getHostName(), ioe); - XTraceContext.logEvent(Fetcher.class, "Fetcher", "Failed failed to shuffle map output: "+ioe.getClass().getName(), + 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 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 b33f0efbd525..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,34 +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.berkeley.xtrace.XTraceContext; +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; @@ -127,7 +122,7 @@ public void shuffle(MapHost host, InputStream input, @Override public void commit() throws IOException { - XTraceContext.logEvent(InMemoryMapOutput.class, "InMemoryMapOutput", "Map output committed"); + 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 202b595cb1d9..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,22 +17,18 @@ */ package org.apache.hadoop.mapreduce.task.reduce; -import java.io.InputStream; import java.io.IOException; - -import java.util.Collection; +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.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; @InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceStability.Unstable @@ -44,7 +40,7 @@ public abstract class MapOutput { private final long size; private final boolean primaryMapOutput; - private Collection xtrace_context; + private Context xtrace_context = null; public MapOutput(TaskAttemptID mapId, long size, boolean primaryMapOutput) { this.id = ID.incrementAndGet(); @@ -54,11 +50,11 @@ public MapOutput(TaskAttemptID mapId, long size, boolean primaryMapOutput) { } public void rememberContext() { - xtrace_context = XTraceContext.getThreadContext(); + xtrace_context = XTrace.get(); } public void joinContext() { - XTraceContext.joinContext(xtrace_context); + XTrace.join(xtrace_context); } public boolean isPrimaryMapOutput() { 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 bda61dc227c3..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 @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -40,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; @@ -61,14 +60,15 @@ import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +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 @@ -258,8 +258,7 @@ public synchronized MapOutput reserve(TaskAttemptID mapId, LOG.info(mapId + ": Shuffling to disk since " + requestedSize + " is greater than maxSingleShuffleLimit (" + maxSingleShuffleLimit + ")"); - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl", - "Shuffling directly to disk due to reservation size exceeding max single shuffle limit", + 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); @@ -284,8 +283,7 @@ public synchronized MapOutput reserve(TaskAttemptID mapId, LOG.debug(mapId + ": Stalling shuffle since usedMemory (" + usedMemory + ") is greater than memoryLimit (" + memoryLimit + ")." + " CommitMemory is (" + commitMemory + ")"); - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl", - "Currently above memory limit, stalling shuffle", "Map ID", mapId, + xtrace.log("Currently above memory limit, stalling shuffle", "Map ID", mapId, "Used Memory", usedMemory, "Memory Limit", memoryLimit, "Commit Memory", commitMemory); return null; } @@ -294,8 +292,7 @@ public synchronized MapOutput reserve(TaskAttemptID mapId, LOG.debug(mapId + ": Proceeding with shuffle since usedMemory (" + usedMemory + ") is lesser than memoryLimit (" + memoryLimit + ")." + "CommitMemory is (" + commitMemory + ")"); - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl", - "Proceeding with shuffle", "Map ID", mapId, "Used Memory", usedMemory, + xtrace.log("Proceeding with shuffle", "Map ID", mapId, "Used Memory", usedMemory, "Memory Limit", memoryLimit, "Commit Memory", commitMemory); return unconditionalReserve(mapId, requestedSize, true); } @@ -407,8 +404,7 @@ public void merge(List> inputs) throws IOException { LOG.info("Initiating Memory-to-Memory merge with " + noInMemorySegments + " segments of total-size: " + mergeOutputSize); - XTraceContext.logEvent(IntermediateMemoryToMemoryMerger.class, "IntermediateMemoryToMemoryMerger start", - "Initiating Memory-to-Memory merge", "Num Segments", noInMemorySegments, "Total Size", mergeOutputSize, + xtrace.log("Initiating Memory-to-Memory merge", "Num Segments", noInMemorySegments, "Total Size", mergeOutputSize, "Reduce ID", reduceId); RawKeyValueIterator rIter = @@ -425,8 +421,7 @@ public void merge(List> inputs) throws IOException { LOG.info(reduceId + " Memory-to-Memory merge of the " + noInMemorySegments + " files in-memory complete."); - XTraceContext.logEvent(IntermediateMemoryToMemoryMerger.class, "IntermediateMemoryToMemoryMerger end", - "Memory-to-Memory merge complete", "Reduce ID", reduceId, "Num Segments", noInMemorySegments); + xtrace.log("Memory-to-Memory merge complete", "Reduce ID", reduceId, "Num Segments", noInMemorySegments); // Note the output of the merge closeInMemoryMergedFile(mergedMapOutputs); @@ -482,8 +477,7 @@ public void merge(List> inputs) throws IOException { try { LOG.info("Initiating in-memory merge with " + noInMemorySegments + " segments..."); - XTraceContext.logEvent(InMemoryMerger.class, "InMemoryMerger start", "Initiating in-memory merge", - "Num Segments", noInMemorySegments); + xtrace.log("Initiating in-memory merge", "Num Segments", noInMemorySegments); rIter = Merger.merge(jobConf, rfs, (Class)jobConf.getMapOutputKeyClass(), @@ -508,7 +502,7 @@ public void merge(List> inputs) throws IOException { " files in-memory complete." + " Local file is " + outputPath + " of size " + localFS.getFileStatus(outputPath).getLen()); - XTraceContext.logEvent(InMemoryMerger.class, "InMemoryMerger","Merge complete", "Num Segments", noInMemorySegments, + 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 @@ -517,7 +511,7 @@ public void merge(List> inputs) throws IOException { throw e; } - XTraceContext.logEvent(InMemoryMerger.class, "InMemoryMerger end", "In-memory merge finished"); + xtrace.log("In-memory merge finished"); // Note the output of the merge closeOnDiskFile(compressAwarePath); @@ -541,7 +535,7 @@ public void merge(List inputs) throws IOException { return; } for (CompressAwarePath input : inputs) { - XTraceContext.joinObject(input); + input.joinContext(); } long approxOutputSize = 0; @@ -552,8 +546,7 @@ public void merge(List inputs) throws IOException { LOG.info("OnDiskMerger: We have " + inputs.size() + " map outputs on disk. Triggering merge..."); - XTraceContext.logEvent(OnDiskMerger.class, "OnDiskMerger start", - "Merging outputs on disk", "Num Segments", inputs.size()); + xtrace.log("Merging outputs on disk", "Num Segments", inputs.size()); // 1. Prepare the list of files to be merged. for (CompressAwarePath file : inputs) { @@ -595,7 +588,7 @@ public void merge(List inputs) throws IOException { throw e; } - XTraceContext.logEvent(OnDiskMerger.class, "OnDiskMerger merge end", "Finished merging map output files on disk", + xtrace.log("Finished merging map output files on disk", "Num Segments", inputs.size(), "Output Size", approxOutputSize, "Output File", outputPath); closeOnDiskFile(compressAwarePath); @@ -703,7 +696,7 @@ 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"); - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge", "Starting final merge", + xtrace.log("Starting final merge", "Num In-Memory Map Outputs", inMemoryMapOutputs.size(), "Num On-Disk Map Outputs", onDiskMapOutputs.size()); final float maxRedPer = @@ -728,7 +721,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, List> memDiskSegments = new ArrayList>(); long inMemToDiskBytes = 0; boolean mergePhaseFinished = false; - Collection start_context = XTraceContext.getThreadContext(); + Context start_context = XTrace.get(); if (inMemoryMapOutputs.size() > 0) { TaskID mapId = inMemoryMapOutputs.get(0).getMapId().getTaskID(); inMemToDiskBytes = createInMemorySegments(inMemoryMapOutputs, @@ -737,8 +730,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, final int numMemDiskSegments = memDiskSegments.size(); if (numMemDiskSegments > 0 && ioSortFactor > onDiskMapOutputs.size()) { - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge flush", - "Merging segments to disk to satisfy reduce memory limit", "Num Segments", numMemDiskSegments, + 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 @@ -761,12 +753,11 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, keyClass, valueClass, codec, null); try { Merger.writeFile(rIter, writer, reporter, job); - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge flush merge", "Merge complete"); + xtrace.log("Merge complete"); writer.close(); CompressAwarePath p = new CompressAwarePath(outputPath, writer.getRawLength(), writer.getCompressedLength()); onDiskMapOutputs.add(p); - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge flush complete", "Merged to disk", - "OutputPath", outputPath.toString()); + xtrace.log("Merged to disk", "OutputPath", outputPath.toString()); p.rememberContext(); writer = null; // add to list of final disk outputs. @@ -789,7 +780,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, "reduce memory limit"); inMemToDiskBytes = 0; memDiskSegments.clear(); - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); } else if (inMemToDiskBytes != 0) { LOG.info("Keeping " + numMemDiskSegments + " segments, " + inMemToDiskBytes + " bytes in memory for " + @@ -797,7 +788,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, } } - start_context = XTraceContext.getThreadContext(); + start_context = XTrace.get(); // segments on disk List> diskSegments = new ArrayList>(); @@ -820,7 +811,7 @@ private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs, } LOG.info("Merging " + onDisk.length + " files, " + onDiskBytes + " bytes from disk"); - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge", "Merging files 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) { @@ -831,8 +822,8 @@ public int compare(Segment o1, Segment o2) { } }); - Collection disk_context = XTraceContext.getThreadContext(); - XTraceContext.setThreadContext(start_context); + 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>(); @@ -841,10 +832,10 @@ public int compare(Segment o1, Segment o2) { LOG.info("Merging " + finalSegments.size() + " segments, " + inMemBytes + " bytes from memory into reduce"); - Collection mem_context = XTraceContext.getThreadContext(); + Context mem_context = XTrace.get(); if (0 != onDiskBytes) { - XTraceContext.setThreadContext(disk_context); + XTrace.set(disk_context); final int numInMemSegments = memDiskSegments.size(); diskSegments.addAll(0, memDiskSegments); memDiskSegments.clear(); @@ -856,12 +847,10 @@ public int compare(Segment o1, Segment o2) { ioSortFactor, numInMemSegments, tmpDir, comparator, reporter, false, spilledRecordsCounter, null, thisPhase); diskSegments.clear(); - XTraceContext.joinContext(mem_context); - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge","Piping segment merge into reduce", - "Num Segments", finalSegments.size(), "Num Bytes", inMemBytes); + XTrace.join(mem_context); + xtrace.log("Piping segment merge into reduce", "Num Segments", finalSegments.size(), "Num Bytes", inMemBytes); if (0 == finalSegments.size()) { - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge complete", - "Final merge complete, returning result iterator"); + xtrace.log("Final merge complete, returning result iterator"); return diskMerge; } finalSegments.add(new Segment( @@ -872,15 +861,14 @@ public int compare(Segment o1, Segment o2) { comparator, reporter, spilledRecordsCounter, null, null); - XTraceContext.logEvent(MergeManagerImpl.class, "MergeManagerImpl finalMerge complete", - "Final merge complete, returning result iterator"); + xtrace.log("Final merge complete, returning result iterator"); return result; } static class CompressAwarePath extends Path { private long rawDataLength; private long compressedSize; - private Collection xtrace_context; + private Context xtrace_context; public CompressAwarePath(Path path, long rawDataLength, long compressSize) { super(path.toUri()); @@ -889,11 +877,11 @@ public CompressAwarePath(Path path, long rawDataLength, long compressSize) { } public void rememberContext() { - xtrace_context = XTraceContext.getThreadContext(); + xtrace_context = XTrace.get(); } public void joinContext() { - XTraceContext.joinContext(xtrace_context); + XTrace.join(xtrace_context); } public long getRawDataLength() { 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 cbf831ddef6a..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,10 +28,11 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import edu.berkeley.xtrace.XTraceContext; +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); @@ -76,7 +77,7 @@ public void startMerge(Set inputs) { public synchronized void waitForMerge() throws InterruptedException { int numPending = this.numPending.get(); while (numPending > 0) { - XTraceContext.logEvent(MergeThread.class, "MergeThread", "Waiting for pending merges", "Num Pending", numPending); + 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 2e6256f16a93..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,30 +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.berkeley.xtrace.XTraceContext; + +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; @@ -135,8 +132,7 @@ public void commit() throws IOException { CompressAwarePath compressAwarePath = new CompressAwarePath(outputPath, getSize(), this.compressedSize); merger.closeOnDiskFile(compressAwarePath); - XTraceContext.logEvent(OnDiskMapOutput.class, "OnDiskMapOutput", "Map output committed", - "tmpOutputPath", tmpOutputPath, "outputPath", outputPath); + 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 762ecc3cea74..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 @@ -18,28 +18,27 @@ package org.apache.hadoop.mapreduce.task.reduce; import java.io.IOException; -import java.util.Collection; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; 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.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +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; @@ -57,7 +56,7 @@ public class Shuffle implements ShuffleConsumerPlugin, ExceptionRepo private MergeManager merger; private Throwable throwable = null; private String throwingThreadName = null; - private Collection throwingContext = null; + private edu.brown.cs.systems.xtrace.Context throwingContext = null; private Progress copyPhase; private TaskStatus taskStatus; private Task reduceTask; //Used for status updates @@ -94,7 +93,7 @@ protected MergeManager createMergeManager( @Override public RawKeyValueIterator run() throws IOException, InterruptedException { - XTraceContext.logEvent(Shuffle.class, "Shuffle", "Running shuffle"); + 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 @@ -125,8 +124,8 @@ public RawKeyValueIterator run() throws IOException, InterruptedException { synchronized (this) { if (throwable != null) { - XTraceContext.joinContext(throwingContext); - XTraceContext.logEvent(Shuffle.class, "Shuffle", "Error during shuffle: "+throwable.getClass().getName(), + 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); @@ -146,7 +145,7 @@ public RawKeyValueIterator run() throws IOException, InterruptedException { // stop the scheduler scheduler.close(); - XTraceContext.logEvent(Shuffle.class, "Shuffle copy complete", "Copy phase complete"); + xtrace.log("Copy phase complete"); copyPhase.complete(); // copy is already complete taskStatus.setPhase(TaskStatus.Phase.SORT); reduceTask.statusUpdate(umbilical); @@ -156,23 +155,22 @@ public RawKeyValueIterator run() throws IOException, InterruptedException { try { kvIter = merger.close(); } catch (Throwable e) { - XTraceContext.logEvent(Shuffle.class, "Shuffle merge error", "Error during final merge: "+e.getClass().getName(), - "Message", e.getMessage()); + 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) { - XTraceContext.joinContext(throwingContext); - XTraceContext.logEvent(Shuffle.class, "Shuffle", "Error during shuffle: "+throwable.getClass().getName(), + 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); } } - XTraceContext.logEvent(Shuffle.class, "Shuffle complete", "Shuffle complete"); + xtrace.log("Shuffle complete"); return kvIter; } @@ -185,7 +183,7 @@ public synchronized void reportException(Throwable t) { if (throwable == null) { throwable = t; throwingThreadName = Thread.currentThread().getName(); - throwingContext = XTraceContext.getThreadContext(); + 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 85c7dbbcfe16..527efa369180 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 @@ -20,7 +20,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.nio.ByteBuffer; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -28,9 +27,9 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; -import edu.berkeley.xtrace.TaskID; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +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 @@ -56,7 +55,7 @@ public class ShuffleHeader implements Writable { long uncompressedLength; long compressedLength; int forReduce; - XTraceMetadata m; + Context m; public ShuffleHeader() { } @@ -73,9 +72,14 @@ public void readFields(DataInput in) throws IOException { compressedLength = WritableUtils.readVLong(in); uncompressedLength = WritableUtils.readVLong(in); forReduce = WritableUtils.readVInt(in); - byte[] taskid = ByteBuffer.allocate(8).putLong(WritableUtils.readVLong(in)).array(); - byte[] opid = ByteBuffer.allocate(8).putLong(WritableUtils.readVLong(in)).array(); - m = new XTraceMetadata(new TaskID(taskid, 8), opid); + long taskid = WritableUtils.readVLong(in); + long opid = WritableUtils.readVLong(in); + if (taskid!=0L) { + XTraceMetadata.Builder builder = XTraceMetadata.newBuilder().setTaskID(taskid); + if (opid!=0L) + builder.addParentEventID(opid); + m = Context.parse(builder.build().toByteArray()); + } } public void write(DataOutput out) throws IOException { @@ -90,7 +94,7 @@ public void write(DataOutput out) throws IOException { private long getXTraceTaskID() { if (m!=null) { try { - return ByteBuffer.wrap(m.getTaskId().get()).getLong(); + return XTraceMetadata.parseFrom(m.bytes()).getTaskID(); } catch (Exception e) { } } @@ -100,7 +104,9 @@ private long getXTraceTaskID() { private long getXTraceOpID() { if (m!=null) { try { - return ByteBuffer.wrap(m.getOpId()).getLong(); + XTraceMetadata xmd = XTraceMetadata.parseFrom(m.bytes()); + if (xmd.getParentEventIDCount() > 0) + return xmd.getParentEventID(0); } catch (Exception e) { } } @@ -109,10 +115,11 @@ private long getXTraceOpID() { public void rememberContext() { - m = XTraceContext.logMerge(); + m = XTrace.get(); } public void joinContext() { - XTraceContext.joinChildProcess(m); +// // TODO: what even is this +// XTraceContext.joinChildProcess(m); } } \ 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 dc2961ceba4d..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,7 +18,6 @@ package org.apache.hadoop.mapreduce.task.reduce; import java.io.IOException; - import java.net.URI; import java.text.DecimalFormat; import java.util.ArrayList; @@ -49,9 +48,8 @@ import org.apache.hadoop.mapreduce.task.reduce.MapHost.State; import org.apache.hadoop.util.Progress; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; @InterfaceAudience.Private @InterfaceStability.Unstable @@ -62,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; @@ -105,7 +104,7 @@ protected Long initialValue() { private final boolean reportReadErrorImmediately; private long maxDelay = MRJobConfig.DEFAULT_MAX_SHUFFLE_FETCH_RETRY_DELAY; - private Collection failure_contexts = new XTraceMetadataCollection(); + private Collection failure_contexts = new HashSet(); public ShuffleSchedulerImpl(JobConf job, TaskStatus status, TaskAttemptID reduceId, @@ -202,8 +201,7 @@ public synchronized void copySucceeded(TaskAttemptID mapId, reduceShuffleBytes.increment(bytes); lastProgressTime = System.currentTimeMillis(); LOG.debug("map " + mapId + " done " + status.getStateString()); - XTraceContext.logEvent(ShuffleScheduler.class, "ShuffleScheduler", - "Shuffle from mapper complete", "Map ID", mapId, "Status", status.getStateString()); + xtrace.log("Shuffle from mapper complete", "Map ID", mapId, "Status", status.getStateString()); } } @@ -239,10 +237,11 @@ public synchronized void copyFailed(TaskAttemptID mapId, MapHost host, } else { hostFailures.put(hostname, new IntWritable(1)); } - failure_contexts = XTraceContext.getThreadContext(failure_contexts); + failure_contexts.add(XTrace.get()); if (failures >= abortFailureLimit) { try { - XTraceContext.joinContext(failure_contexts); + for (Context ctx : failure_contexts) + XTrace.join(ctx); throw new IOException(failures + " failures downloading " + mapId); } catch (IOException ie) { reporter.reportException(ie); @@ -379,7 +378,7 @@ public synchronized MapHost getHost() throws InterruptedException { LOG.info("Assigning " + host + " with " + host.getNumKnownMapOutputs() + " to " + Thread.currentThread().getName()); - XTraceContext.logEvent(ShuffleScheduler.class, "ShuffleScheduler", "Selected a host for shuffle", + xtrace.log("Selected a host for shuffle", "Host", host, "Num Outputs", host.getNumKnownMapOutputs(), "Thread Name", Thread.currentThread().getName()); shuffleStart.set(System.currentTimeMillis()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java index 6d66c3852e31..a88e8549427f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java @@ -67,9 +67,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - /** * This class provides a way to interact with history files in a thread safe * manor. @@ -748,18 +745,15 @@ private void scanIntermediateDirectory(final Path absPath) throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("Scheduling move to done of " +found); } - final Collection xtrace_context = XTraceContext.getThreadContext(); moveToDoneExecutor.execute(new Runnable() { @Override public void run() { - XTraceContext.setThreadContext(xtrace_context); try { found.moveToDone(); } catch (IOException e) { LOG.info("Failed to process fileInfo for job: " + found.getJobId(), e); } - XTraceContext.clearThreadContext(); } }); } 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 176dd4cb61d3..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 @@ -19,7 +19,6 @@ package org.apache.hadoop.mapreduce.v2.hs; import java.io.IOException; -import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -53,9 +52,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - /** * Loads and manages the Job history cache. */ @@ -177,35 +173,20 @@ public String getApplicationName() { private class MoveIntermediateToDoneRunnable implements Runnable { - private Collection xtrace_context; - - public MoveIntermediateToDoneRunnable() { - this.xtrace_context = XTraceContext.getThreadContext(); - } - @Override public void run() { - XTraceContext.setThreadContext(xtrace_context); try { LOG.info("Starting scan to move intermediate done files"); hsManager.scanIntermediateDirectory(); } catch (IOException e) { LOG.error("Error while scanning intermediate done dir ", e); } - XTraceContext.clearThreadContext(); } } private class HistoryCleaner implements Runnable { - private Collection xtrace_context; - - public HistoryCleaner() { - this.xtrace_context = XTraceContext.getThreadContext(); - } - public void run() { - XTraceContext.setThreadContext(xtrace_context); LOG.info("History Cleaner started"); try { hsManager.clean(); @@ -213,7 +194,6 @@ public void run() { LOG.warn("Error trying to clean up ", e); } LOG.info("History Cleaner complete"); - XTraceContext.clearThreadContext(); } } 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 d38699e5f5a3..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; @@ -56,11 +56,13 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.util.ConverterUtils; -import edu.berkeley.xtrace.XTraceContext; 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; @@ -172,7 +174,7 @@ public JobID getNewJobID() throws IOException, InterruptedException { this.application = client.createApplication().getApplicationSubmissionContext(); this.applicationId = this.application.getApplicationId(); JobID id = TypeConverter.fromYarn(applicationId); - XTraceContext.logEvent("ResourceMgrDelegate", "Job ID acquired", "Job ID", id.getId()); + 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-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 c8100185f697..515d15d6bca5 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; @@ -40,7 +39,6 @@ import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -114,11 +112,13 @@ import com.google.common.base.Charsets; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; + +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); public static final String SHUFFLE_MANAGE_OS_CACHE = "mapreduce.shuffle.manage.os.cache"; @@ -451,9 +451,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) } String xtrace_context = request.getHeader("X-Trace"); - if (xtrace_context!=null) { - XTraceContext.setThreadContext(XTraceMetadata.createFromString(xtrace_context)); - } + if (xtrace_context!=null) + XTrace.set(xtrace_context); final Map> q = new QueryStringDecoder(request.getUri()).getParameters(); @@ -466,8 +465,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) "\n reduceId: " + reduceQ + "\n jobId: " + jobQ); } - XTraceContext.logEvent(ShuffleHandler.class, "ShuffleHandler", - "Handling map output retrieval request", "URI", request.getUri(), "Map IDs", mapIds, + 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) { @@ -510,13 +508,13 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) ch.write(response); // TODO refactor the following into the pipeline ChannelFuture lastMap = null; - Collection start_context = XTraceContext.getThreadContext(); + Context start_context = XTrace.get(); for (String mapId : mapIds) { try { lastMap = sendMapOutput(ctx, ch, userRsrc.get(jobId), jobId, mapId, reduceId); if (null == lastMap) { - XTraceContext.logEvent(ShuffleHandler.class, "ShuffleHandler", "Error: "+NOT_FOUND); + xtrace.log("Error: "+NOT_FOUND); sendError(ctx, NOT_FOUND); return; } @@ -528,12 +526,11 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) sb.append(t.getCause().getMessage()); t = t.getCause(); } - XTraceContext.logEvent(ShuffleHandler.class, "ShuffleHandler", "Shuffle error: "+e.getClass().getName(), - "Message", e.getMessage()); + xtrace.log("Shuffle error: "+e.getClass().getName(), "Message", e.getMessage()); sendError(ctx,sb.toString() , INTERNAL_SERVER_ERROR); return; } - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); } lastMap.addListener(metrics); lastMap.addListener(ChannelFutureListener.CLOSE); @@ -609,8 +606,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, final IndexRecord info = indexCache.getIndexInformation(mapId, reduce, indexFileName, user); info.joinContext(); - XTraceContext.logEvent(ShuffleHandler.class, "ShuffleHandler", - "Sending map output", "Reduce", reduce, "Map ID", mapId, "Job ID", jobID); + xtrace.log("Sending map output", "Reduce", reduce, "Map ID", mapId, "Job ID", jobID); final ShuffleHeader header = new ShuffleHeader(mapId, info.partLength, info.rawLength, reduce); header.rememberContext(); @@ -668,7 +664,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", XTraceContext.logMerge()); + 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-project-dist/pom.xml b/hadoop-project-dist/pom.xml index 6026c6b58d9c..2f5915be1e85 100644 --- a/hadoop-project-dist/pom.xml +++ b/hadoop-project-dist/pom.xml @@ -55,9 +55,9 @@ 1.7.3 - edu.berkeley.xtrace + edu.brown.cs.systems xtrace - 2.1-20120824 + 3.0 edu.brown.cs.systems 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 c4e67773f70b..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,7 +27,7 @@ import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.ShutdownHookManager; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** * This class is intended to be installed by calling @@ -41,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 @@ -51,7 +52,7 @@ public void uncaughtException(Thread t, Throwable e) { } else if(e instanceof Error) { try { LOG.fatal("Thread " + t + " threw an Error. Shutting down now...", e); - XTraceContext.logEvent(UncaughtExceptionHandler.class, "UncaughtError", e.getClass().getName(), + 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 @@ -61,19 +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..."); - XTraceContext.joinParentProcess(); +// // TODO: XTrace do this +// XTraceContext.joinParentProcess(); } catch (Throwable err) { //Again we done want to exit because of logging issues. } ExitUtil.halt(-1); } else { - XTraceContext.joinParentProcess(); +// // TODO: XTrace do this +// XTraceContext.joinParentProcess(); ExitUtil.terminate(-1); } } else { LOG.error("Thread " + t + " threw an Exception.", e); - XTraceContext.logEvent(UncaughtExceptionHandler.class, "UncaughtException", e.getClass().getName(), - "Thread Name", t.getName(), "Message", e.getMessage()); + 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 d0d67a0c4ac9..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 @@ -20,15 +20,16 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import com.google.protobuf.ByteString; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; import com.google.common.base.Preconditions; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import com.google.protobuf.ByteString; + +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; @Private @Unstable @@ -36,7 +37,7 @@ public class ContainerIdPBImpl extends ContainerId { ContainerIdProto proto = null; ContainerIdProto.Builder builder = null; private ApplicationAttemptId applicationAttemptId = null; - XTraceMetadata xmd = null; + Context xmd = null; public ContainerIdPBImpl() { builder = ContainerIdProto.newBuilder(); @@ -46,10 +47,7 @@ public ContainerIdPBImpl(ContainerIdProto proto) { this.proto = proto; this.applicationAttemptId = convertFromProtoFormat(proto.getAppAttemptId()); if (proto!=null && proto.hasXtrace()) { - ByteString xbs = proto.getXtrace(); - xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); - if (!xmd.isValid()) - xmd = null; + xmd = Context.parse(proto.getXtrace().toByteArray()); } } @@ -86,17 +84,12 @@ protected void setApplicationAttemptId(ApplicationAttemptId atId) { @Override public void rememberContext() { - XTraceMetadata ctx = XTraceContext.logMerge(); - if (ctx!=null && ctx.isValid()) { - xmd = ctx; - } else { - xmd = null; - } + xmd = XTrace.get(); } @Override public void joinContext() { - XTraceContext.joinContext(xmd); + XTrace.join(xmd); } private ApplicationAttemptIdPBImpl convertFromProtoFormat( @@ -111,9 +104,9 @@ private ApplicationAttemptIdProto convertToProtoFormat( @Override protected void build() { - if (xmd!=null && xmd.isValid() && builder!=null) { - builder.setXtrace(ByteString.copyFrom(xmd.pack())); - } + 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 93ca4c29fd3a..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 @@ -20,7 +20,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import com.google.protobuf.ByteString; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; @@ -31,9 +30,10 @@ 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.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; + +import edu.brown.cs.systems.xtrace.XTrace; @Private @Unstable @@ -198,22 +198,15 @@ public synchronized void setPattern(String pattern) { @Override public void rememberContext() { maybeInitBuilder(); - XTraceMetadata ctx = XTraceContext.logMerge(); - if (ctx!=null && ctx.isValid()) { - builder.setXtrace(ByteString.copyFrom(ctx.pack())); - } + if (XTrace.active()) + builder.setXtrace(ByteString.copyFrom(XTrace.bytes())); } @Override public void joinContext() { LocalResourceProtoOrBuilder p = viaProto ? proto : builder; - if (p.hasXtrace()) { - ByteString xbs = p.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); - if (xmd.isValid()) { - XTraceContext.joinContext(xmd); - } - } + if (p.hasXtrace()) + XTrace.join(p.getXtrace().toByteArray()); } private LocalResourceTypeProto convertToProtoFormat(LocalResourceType 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 71200fa6d996..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 @@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import com.google.protobuf.ByteString; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -30,8 +29,9 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProtoOrBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import com.google.protobuf.ByteString; + +import edu.brown.cs.systems.xtrace.XTrace; @Private @Unstable @@ -170,22 +170,15 @@ public void setRelaxLocality(boolean relaxLocality) { @Override public void rememberContext() { maybeInitBuilder(); - XTraceMetadata ctx = XTraceContext.logMerge(); - if (ctx!=null && ctx.isValid()) { - builder.setXtrace(ByteString.copyFrom(ctx.pack())); - } + if (XTrace.active()) + builder.setXtrace(ByteString.copyFrom(XTrace.bytes())); } @Override public void joinContext() { ResourceRequestProtoOrBuilder p = viaProto ? proto : builder; - if (p.hasXtrace()) { - ByteString xbs = p.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); - if (xmd.isValid()) { - XTraceContext.joinContext(xmd); - } - } + if (p.hasXtrace()) + XTrace.join(p.getXtrace().toByteArray()); } private PriorityPBImpl convertFromProtoFormat(PriorityProto 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 2e24cefe0ffe..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 @@ -20,9 +20,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; -import java.util.Collection; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; + +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. @@ -34,7 +34,7 @@ public abstract class AbstractEvent> private final TYPE type; private final long timestamp; - private Collection xtraceContext; + private Context xtraceContext; // use this if you DON'T care about the timestamp public AbstractEvent(TYPE type) { @@ -70,12 +70,12 @@ public String toString() { */ @Override public void rememberContext() { - this.xtraceContext = XTraceContext.getThreadContext(); + this.xtraceContext = XTrace.get(); } @Override public void joinContext() { - XTraceContext.joinContext(this.xtraceContext); + 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 48ec2637ec99..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 @@ -21,6 +21,7 @@ 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; @@ -35,9 +36,8 @@ import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +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 @@ -49,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; @@ -73,7 +74,7 @@ Runnable createThread() { @Override public void run() { while (!stopped && !Thread.currentThread().isInterrupted()) { - XTraceContext.clearThreadContext(); + XTrace.stop(); Event event; try { event = eventQueue.take(); @@ -150,8 +151,9 @@ protected void dispatch(Event event) { if (exitOnDispatchException && (ShutdownHookManager.get().isShutdownInProgress()) == false) { LOG.info("Exiting, bbye.."); - XTraceContext.logEvent(AsyncDispatcher.class, "AsyncDispatcher", "Exiting, bbye.."); - XTraceContext.joinParentProcess(); + xtrace.log("Exiting, bbye.."); +// // TODO: XTrace deal with this +// XTraceContext.joinParentProcess(); System.exit(-1); } } @@ -223,15 +225,16 @@ public MultiListenerHandler() { @Override public void handle(Event event) { - Collection start_xtrace_context = XTraceContext.getThreadContext(); - Collection result_xtrace_contexts = new XTraceMetadataCollection(); + Context start_xtrace_context = XTrace.get(); + Collection result_xtrace_contexts = new HashSet(); for (EventHandler handler: listofHandlers) { - XTraceContext.setThreadContext(start_xtrace_context); + XTrace.set(start_xtrace_context); event.joinContext(); handler.handle(event); - result_xtrace_contexts = XTraceContext.getThreadContext(result_xtrace_contexts); + result_xtrace_contexts.add(XTrace.get()); } - XTraceContext.setThreadContext(result_xtrace_contexts); + 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/factories/impl/pb/RpcClientFactoryPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java index f644a2aef532..e7c737cbb489 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java @@ -31,7 +31,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RpcClientFactory; -import edu.berkeley.xtrace.XTraceContext; @Private public class RpcClientFactoryPBImpl implements RpcClientFactory { 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 4f5620d91633..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 @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.state; -import java.util.Collection; import java.util.EnumMap; import java.util.HashMap; import java.util.Iterator; @@ -29,11 +28,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; -import org.apache.hadoop.yarn.state.StateMachineFactory.Trace; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceEvent; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** * State machine topology. @@ -463,7 +460,7 @@ private boolean logTransitionAsXTraceProcess(STATE oldState, EVENTTYPE eventType = transitionMap.get(eventType); if (transition != null) { Trace trace_type = transition.getTraceType(); - return trace_type==Trace.ALWAYS || (trace_type==Trace.KEEPALIVE && !XTraceContext.isValid()); + return trace_type==Trace.ALWAYS || (trace_type==Trace.KEEPALIVE && !XTrace.active()); } } return false; @@ -612,22 +609,24 @@ private String xtraceStateName(STATE state) { private class InternalStateMachine implements StateMachine { + private final XTrace.Logger xtrace; private final OPERAND operand; private STATE currentState; private String state_machine_id; - private Collection previous_transition_context; + 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; - XTraceContext.logEvent(operand.getClass(), operand.getClass().getSimpleName()+" init", "StateMachine initialized", + xtrace.log(operand.getClass().getSimpleName() + " StateMachine initialized", "StartState", xtraceStateName(currentState), "StateMachineID", state_machine_id); - this.previous_transition_context = XTraceContext.getThreadContext(); + this.previous_transition_context = XTrace.get(); } @Override @@ -636,7 +635,7 @@ public synchronized STATE getCurrentState() { } public void joinPreviousTransitionXTraceContext() { - XTraceContext.joinContext(this.previous_transition_context); + XTrace.join(this.previous_transition_context); } public synchronized STATE doTransition(EVENTTYPE eventType, EVENT event) @@ -649,7 +648,7 @@ public synchronized STATE doTransition(EVENTTYPE eventType, EVENT event) } // Create an event for the transition - XTraceContext.logEvent(operand.getClass(), operand.getClass().getSimpleName()+event.toString(), event.toString(), + xtrace.log(operand.getClass().getSimpleName()+event.toString(), "StartState", xtraceStateName(currentState), "Operand", operand.toString(), "StateMachineID", state_machine_id); set_previous_transition_context = true; @@ -662,8 +661,8 @@ public synchronized STATE doTransition(EVENTTYPE eventType, EVENT event) throw e; } - if (set_previous_transition_context && XTraceContext.isValid()) { - this.previous_transition_context = XTraceContext.getThreadContext(); + 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 39522d913e7a..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 @@ -23,8 +23,6 @@ import java.io.IOException; import java.net.URISyntaxException; import java.security.PrivilegedExceptionAction; -import java.util.Collection; -import java.util.Random; import java.util.concurrent.Callable; import java.util.regex.Pattern; @@ -45,8 +43,8 @@ import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** * Download a single URL to the local disk. @@ -55,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; @@ -65,7 +64,7 @@ public class FSDownload implements Callable { /** The local FS dir path under which this resource is to be localized to */ private Path destDirPath; - private Collection xtrace_context; + private Context xtrace_context; private static final FsPermission cachePerms = new FsPermission( (short) 0755); @@ -83,7 +82,7 @@ public FSDownload(FileContext files, UserGroupInformation ugi, Configuration con this.files = files; this.userUgi = ugi; this.resource = resource; - this.xtrace_context = XTraceContext.getThreadContext(); + this.xtrace_context = XTrace.get(); } LocalResource getResource() { @@ -177,7 +176,7 @@ private static boolean checkPermissionOfOther(FileSystem fs, Path path, private Path copy(Path sCopy, Path dstdir) throws IOException { - XTraceContext.logEvent(FSDownload.class, "FSDownload", "Copying", "Source", sCopy.toString(), "Destination", dstdir.toString()); + 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); @@ -199,7 +198,7 @@ private Path copy(Path sCopy, Path dstdir) throws IOException { } private long unpack(File localrsrc, File dst, Pattern pattern) throws IOException { - XTraceContext.logEvent(FSDownload.class, "FSDownload", "Unpacking", "Resource", localrsrc.toString(), "Unpack To", dst.toString()); + xtrace.log("Unpacking", "Resource", localrsrc.toString(), "Unpack To", dst.toString()); switch (resource.getType()) { case ARCHIVE: { String lowerDst = dst.getName().toLowerCase(); @@ -272,13 +271,13 @@ private long unpack(File localrsrc, File dst, Pattern pattern) throws IOExceptio @Override public Path call() throws Exception { - XTraceContext.setThreadContext(xtrace_context); - XTraceContext.logEvent(FSDownload.class, "FSDownload", "Localizing resource", "Resource",resource.getResource().toString()); + XTrace.set(xtrace_context); + xtrace.log("Localizing resource", "Resource",resource.getResource().toString()); final Path sCopy; try { sCopy = ConverterUtils.getPathFromYarnURL(resource.getResource()); } catch (URISyntaxException e) { - XTraceContext.logEvent(FSDownload.class, "FSDownload", "Invalid resource: "+e.getClass().getName(), "Message", e.getMessage()); + xtrace.log("Invalid resource: "+e.getClass().getName(), "Message", e.getMessage()); throw new IOException("Invalid resource", e); } createDir(destDirPath, cachePerms); @@ -315,8 +314,9 @@ public Path run() throws Exception { resource = null; } Path p = files.makeQualified(new Path(destDirPath, sCopy.getName())); - XTraceContext.rememberObject(p); - XTraceContext.clearThreadContext(); +// // 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-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 9c2455378c0a..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 @@ -52,10 +52,12 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; + +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); @@ -188,26 +190,23 @@ public int launchContainer(Container container, containerIdStr, this.getConf()); LOG.info("launchContainer: " + Arrays.toString(command)); - XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor", "Invoking command line", - "args", Arrays.toString(command), "Container ID", containerIdStr); + 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(); - XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor", "Subprocess finished with exit code "+shExec.getExitCode()); + xtrace.log("Subprocess finished with exit code "+shExec.getExitCode()); } else { LOG.info("Container " + containerIdStr + " was marked as inactive. Returning terminated error"); - XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor", - "Container 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) { - XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor IOException: " + e.getClass().getName(), - "Message", e.getMessage()); + xtrace.log("DefaultContainerExecutor IOException: " + e.getClass().getName(), "Message", e.getMessage()); return -1; } int exitCode = shExec.getExitCode(); @@ -228,7 +227,7 @@ public int launchContainer(Container container, container.handle(new ContainerDiagnosticsUpdateEvent(containerId, "Container killed on request. Exit code is " + exitCode)); } - XTraceContext.logEvent(ContainerExecutor.class, "DefaultContainerExecutor", "Subprocess finished with exit code "+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 fc0b74f695d0..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 @@ -26,7 +26,6 @@ import java.util.Iterator; import java.util.List; import java.util.Set; -import java.util.Collection; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; @@ -45,9 +44,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - public class DeletionService extends AbstractService { static final Log LOG = LogFactory.getLog(DeletionService.class); private int debugDelay; @@ -154,8 +150,6 @@ public static class FileDeletionTask implements Runnable { // fileDeletionTaskFinished(). private boolean success; - private Collection xtrace_context; - private FileDeletionTask(DeletionService delService, String user, Path subDir, List baseDirs) { this.delService = delService; @@ -165,8 +159,6 @@ private FileDeletionTask(DeletionService delService, String user, this.successorTaskSet = new HashSet(); this.numberOfPendingPredecessorTasks = new AtomicInteger(0); success = true; - - this.xtrace_context = XTraceContext.getThreadContext(); } /** @@ -212,7 +204,6 @@ public void run() { LOG.debug(this); } boolean error = false; - XTraceContext.joinContext(xtrace_context); if (null == user) { if (baseDirs == null || baseDirs.size() == 0) { LOG.debug("NM deleting absolute path : " + subDir); @@ -255,7 +246,6 @@ public void run() { setSuccess(!error); } fileDeletionTaskFinished(); - XTraceContext.clearThreadContext(); } @Override 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 a7ae13c0b164..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,10 +43,11 @@ import org.apache.hadoop.yarn.server.nodemanager.util.LCEResourcesHandler; import org.apache.hadoop.yarn.util.ConverterUtils; -import edu.berkeley.xtrace.XTraceContext; +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); @@ -145,7 +146,7 @@ public void init() throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("checkLinuxExecutorSetup: " + Arrays.toString(commandArray)); } - XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "init", "args", Arrays.toString(commandArray)); + xtrace.log("LinuxContainerExecutor init", "args", Arrays.toString(commandArray)); try { shExec.execute(); } catch (ExitCodeException e) { @@ -201,7 +202,7 @@ public void startLocalizer(Path nmPrivateContainerTokensPath, if (LOG.isDebugEnabled()) { LOG.debug("initApplication: " + Arrays.toString(commandArray)); } - XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "Starting localizer", "args", Arrays.toString(commandArray)); + xtrace.log("Starting localizer", "args", Arrays.toString(commandArray)); try { shExec.execute(); if (LOG.isDebugEnabled()) { @@ -212,7 +213,7 @@ public void startLocalizer(Path nmPrivateContainerTokensPath, LOG.warn("Exit code from container " + locId + " startLocalizer is : " + exitCode, e); logOutput(shExec.getOutput()); - XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "Container finished with exit code "+exitCode); + xtrace.log("Container finished with exit code "+exitCode); throw new IOException("Application " + appId + " initialization failed" + " (exitCode=" + exitCode + ") with output: " + shExec.getOutput(), e); } @@ -254,7 +255,7 @@ public int launchContainer(Container container, container.getLaunchContext().getEnvironment()); // sanitized env // DEBUG LOG.info("launchContainer: " + Arrays.toString(commandArray)); - XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "Launching container", "args", Arrays.toString(commandArray)); + xtrace.log("Launching container", "args", Arrays.toString(commandArray)); shExec.execute(); if (LOG.isDebugEnabled()) { logOutput(shExec.getOutput()); @@ -269,7 +270,7 @@ public int launchContainer(Container container, } int exitCode = shExec.getExitCode(); LOG.warn("Exit code from container " + containerId + " is : " + exitCode); - XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "Container Finished with exit code "+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 @@ -311,7 +312,7 @@ public boolean signalContainer(String user, String pid, Signal signal) if (LOG.isDebugEnabled()) { LOG.debug("signalContainer: " + Arrays.toString(command)); } - XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "signalContainer: Signalling container with signal " + signal.toString(), "args", Arrays.toString(command)); + xtrace.log("signalContainer: Signalling container with signal " + signal.toString(), "args", Arrays.toString(command)); try { shExec.execute(); } catch (ExitCodeException e) { @@ -322,7 +323,7 @@ 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()); - XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "signalContainer: Problem signalling container, exit code "+ret_code, + 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: " @@ -353,7 +354,7 @@ public void deleteAsUser(String user, Path dir, Path... baseDirs) { if (LOG.isDebugEnabled()) { LOG.debug("deleteAsUser: " + Arrays.toString(commandArray)); } - XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "Delete as user ", "args", Arrays.toString(commandArray)); + xtrace.log("Delete as user ", "args", Arrays.toString(commandArray)); try { shExec.execute(); if (LOG.isDebugEnabled()) { @@ -365,7 +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()); - XTraceContext.logEvent(ContainerExecutor.class, "LinuxContainerExecutor", "deleteAsUser exit code from container is: " + exitCode); + 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 8c3fc5fee1fa..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 @@ -21,7 +21,6 @@ import java.io.IOException; import java.net.ConnectException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -66,8 +65,7 @@ import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; public class NodeStatusUpdaterImpl extends AbstractService implements @@ -358,7 +356,7 @@ protected void startStatusUpdater() { public void run() { int lastHeartBeatID = 0; while (!isStopped) { - XTraceContext.clearThreadContext(); + 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/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 c5917abb32cc..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 @@ -17,11 +17,6 @@ */ package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb; -import com.google.protobuf.ByteString; - -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.SerializedException; import org.apache.hadoop.yarn.api.records.URL; @@ -38,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 { @@ -229,22 +228,15 @@ private SerializedExceptionProto convertToProtoFormat(SerializedException t) { @Override public void rememberContext() { maybeInitBuilder(); - XTraceMetadata ctx = XTraceContext.logMerge(); - if (ctx!=null && ctx.isValid()) { - builder.setXtrace(ByteString.copyFrom(ctx.pack())); - } + if (XTrace.active()) + builder.setXtrace(ByteString.copyFrom(XTrace.bytes())); } @Override public void joinContext() { LocalResourceStatusProtoOrBuilder p = viaProto ? proto : builder; - if (p.hasXtrace()) { - ByteString xbs = p.getXtrace(); - XTraceMetadata xmd = XTraceMetadata.createFromBytes(xbs.toByteArray(), 0, xbs.size()); - if (xmd.isValid()) { - XTraceContext.joinContext(xmd); - } - } + 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/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 ff5903f479f4..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,12 +115,13 @@ import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; +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; @@ -456,9 +457,7 @@ private void startContainerInternal(NMTokenIdentifier nmTokenIdentifier, String user = containerTokenIdentifier.getApplicationSubmitter(); LOG.info("Start request for " + containerIdStr + " by user " + user); - - XTraceContext.logEvent(ContainerManagerImpl.class, "ContainerManagerImpl", "Starting container", - "User", user, "Container ID", containerIdStr); + xtrace.log("Starting container", "User", user, "Container ID", containerIdStr); ContainerLaunchContext launchContext = request.getContainerLaunchContext(); @@ -728,7 +727,7 @@ public void handle(ContainerManagerEvent event) { case FINISH_CONTAINERS: CMgrCompletedContainersEvent containersFinishedEvent = (CMgrCompletedContainersEvent) event; - XTraceContext.clearThreadContext(); + XTrace.stop(); for (ContainerId container : containersFinishedEvent .getContainersToCleanup()) { container.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/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 4d76cf10de64..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 @@ -21,6 +21,7 @@ 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; @@ -53,9 +54,7 @@ import org.apache.hadoop.yarn.state.StateMachine; import org.apache.hadoop.yarn.state.StateMachineFactory; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +import edu.brown.cs.systems.xtrace.XTrace; /** * The state machine for the representation of an Application @@ -304,15 +303,16 @@ static class AppInitDoneTransition implements @Override public void transition(ApplicationImpl app, ApplicationEvent event) { // Start all the containers waiting for ApplicationInit - Collection start_context = XTraceContext.getThreadContext(); - Collection end_context = new XTraceMetadataCollection(); + edu.brown.cs.systems.xtrace.Context start_context = XTrace.get(); + Collection end_contexts = new HashSet(); for (Container container : app.containers.values()) { - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); app.dispatcher.getEventHandler().handle(new ContainerInitEvent( container.getContainerId())); - end_context = XTraceContext.getThreadContext(end_context); + end_contexts.add(XTrace.get()); } - XTraceContext.joinContext(end_context); + for (edu.brown.cs.systems.xtrace.Context ctx : end_contexts) + XTrace.join(ctx); } } 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 5e5404b8a720..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,9 +70,8 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.ConverterUtils; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; public class ContainerImpl implements Container { @@ -104,7 +103,7 @@ public class ContainerImpl implements Container { private final List appRsrcs = new ArrayList(); - private Collection xtrace_localizedresources = new XTraceMetadataCollection(); + private Collection xtrace_localizedresources = new HashSet(); public ContainerImpl(Configuration conf, Dispatcher dispatcher, ContainerLaunchContext launchContext, Credentials creds, @@ -605,11 +604,12 @@ public ContainerState transition(ContainerImpl container, return ContainerState.LOCALIZING; } container.localizedResources.put(rsrcEvent.getLocation(), syms); - container.xtrace_localizedresources = XTraceContext.getThreadContext(container.xtrace_localizedresources); + container.xtrace_localizedresources.add(XTrace.get()); if (!container.pendingResources.isEmpty()) { return ContainerState.LOCALIZING; } - XTraceContext.setThreadContext(container.xtrace_localizedresources); + for (Context ctx : container.xtrace_localizedresources) + XTrace.join(ctx); container.dispatcher.getEventHandler().handle( new ContainersLauncherEvent(container, ContainersLauncherEventType.LAUNCH_CONTAINER)); 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/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java index 7dd5facd3eff..597b92102f45 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java @@ -27,7 +27,6 @@ import java.io.OutputStream; import java.io.PrintStream; import java.util.ArrayList; -import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; import java.util.List; @@ -72,8 +71,7 @@ import org.apache.hadoop.yarn.util.Apps; import org.apache.hadoop.yarn.util.ConverterUtils; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; public class ContainerLaunch implements Callable { @@ -102,7 +100,7 @@ public class ContainerLaunch implements Callable { private final LocalDirsHandlerService dirsHandler; - private Collection xtrace; + private edu.brown.cs.systems.xtrace.Context xtrace_context; public ContainerLaunch(Context context, Configuration configuration, Dispatcher dispatcher, ContainerExecutor exec, Application app, @@ -120,14 +118,14 @@ public ContainerLaunch(Context context, Configuration configuration, this.maxKillWaitTime = conf.getLong(YarnConfiguration.NM_PROCESS_KILL_WAIT_MS, YarnConfiguration.DEFAULT_NM_PROCESS_KILL_WAIT_MS); - this.xtrace = XTraceContext.getThreadContext(); + xtrace_context = XTrace.get(); } @Override @SuppressWarnings("unchecked") // dispatcher not typed public Integer call() { - XTraceContext.setThreadContext(this.xtrace); + XTrace.set(xtrace_context); final ContainerLaunchContext launchContext = container.getLaunchContext(); final Map> localResources = 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/launcher/ContainersLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java index d116bfec7a30..815772aded09 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java @@ -45,7 +45,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** * The launcher for the containers. This service should be started only after @@ -56,6 +56,7 @@ public class ContainersLauncher extends AbstractService implements EventHandler { + private static final XTrace.Logger xtrace = XTrace.getLogger(ContainersLauncher.class); private static final Log LOG = LogFactory.getLog(ContainersLauncher.class); private final Context context; @@ -117,7 +118,7 @@ public void handle(ContainersLauncherEvent event) { ContainerId containerId = container.getContainerId(); switch (event.getType()) { case LAUNCH_CONTAINER: - XTraceContext.logEvent(ContainersLauncher.class, "ContainersLauncher", "Launching container"); + xtrace.log("Launching container"); Application app = context.getApplications().get( containerId.getApplicationAttemptId().getApplicationId()); @@ -130,7 +131,7 @@ public void handle(ContainersLauncherEvent event) { launch)); break; case CLEANUP_CONTAINER: - XTraceContext.logEvent(ContainersLauncher.class, "ContainersLauncher", "Cleanup container"); + xtrace.log("Cleanup container"); RunningContainer rContainerDatum = running.remove(containerId); if (rContainerDatum == null) { // Container not launched. So nothing needs to be done. 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/localizer/ContainerLocalizer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java index 6ef0278d058d..33f9769c249c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java @@ -24,7 +24,6 @@ import java.security.PrivilegedAction; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -66,17 +65,16 @@ import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus; import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenIdentifier; -import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.FSDownload; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; public class ContainerLocalizer { + static final XTrace.Logger xtrace = XTrace.getLogger(ContainerLocalizer.class); static final Log LOG = LogFactory.getLog(ContainerLocalizer.class); public static final String FILECACHE = "filecache"; @@ -229,7 +227,7 @@ protected void localizeFiles(LocalizationProtocol nodemanager, CompletionService cs, UserGroupInformation ugi) throws IOException { while (true) { - XTraceContext.clearThreadContext(); + XTrace.stop(); try { LocalizerStatus status = createStatus(); LocalizerHeartbeatResponse response = nodemanager.heartbeat(status); @@ -243,7 +241,7 @@ protected void localizeFiles(LocalizationProtocol nodemanager, new Path(newRsrc.getDestinationDirectory().getFile()), newRsrc.getResource(), ugi))); } - XTraceContext.clearThreadContext(); + XTrace.stop(); } break; case DIE: @@ -289,7 +287,8 @@ private LocalizerStatus createStatus() throws InterruptedException { if (fPath.isDone()) { try { Path localPath = fPath.get(); - XTraceContext.joinObject(localPath); +// // TODO: deal with this properly +// XTraceContext.joinObject(localPath); stat.rememberContext(); stat.setLocalPath( ConverterUtils.getYarnUrlFromPath(localPath)); @@ -309,7 +308,7 @@ private LocalizerStatus createStatus() throws InterruptedException { stat.setStatus(ResourceStatusType.FETCH_PENDING); } currentResources.add(stat); - XTraceContext.clearThreadContext(); + XTrace.stop(); } LocalizerStatus status = recordFactory.newRecordInstance(LocalizerStatus.class); @@ -327,7 +326,7 @@ public static void main(String[] argv) throws Throwable { // MKDIR $x/$user/appcache/$appid/filecache // LOAD $x/$user/appcache/$appid/appTokens try { - XTraceContext.startTrace("ContainerLocalizer", "ContainerLocalizer launched..."); + xtrace.log("ContainerLocalizer launched..."); String user = argv[0]; String appId = argv[1]; String locId = argv[2]; @@ -351,7 +350,8 @@ public static void main(String[] argv) throws Throwable { appId, locId, localDirs, RecordFactoryProvider.getRecordFactory(null)); int retCode = localizer.runLocalization(nmAddr); - XTraceContext.joinParentProcess(); +// // TODO: Xtrace deal with this +// XTraceContext.joinParentProcess(); System.exit(retCode); } catch (Throwable e) { // Print error to stdout so that LCE can use it. 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/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java index 2b92348ff87d..a89f0aeae42f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java @@ -31,6 +31,7 @@ import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -81,8 +82,8 @@ import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; -import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService; import org.apache.hadoop.yarn.server.nodemanager.DeletionService.FileDeletionTask; +import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService; import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol; import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec; import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus; @@ -119,13 +120,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; public class ResourceLocalizationService extends CompositeService implements EventHandler, LocalizationProtocol { + private static final XTrace.Logger xtrace = XTrace.getLogger(ResourceLocalizationService.class); private static final Log LOG = LogFactory.getLog(ResourceLocalizationService.class); public static final String NM_PRIVATE_DIR = "nmPrivate"; public static final FsPermission NM_PRIVATE_PERM = new FsPermission((short) 0700); @@ -369,8 +370,8 @@ private void handleInitContainerResources( c.getUser(), c.getContainerId(), c.getCredentials()); Map> rsrcs = rsrcReqs.getRequestedResources(); - Collection startCtx = XTraceContext.getThreadContext(); - Collection endCtxs = new XTraceMetadataCollection(); + Context startCtx = XTrace.get(); + Collection endCtxs = new HashSet(); for (Map.Entry> e : rsrcs.entrySet()) { LocalResourcesTracker tracker = @@ -378,12 +379,13 @@ private void handleInitContainerResources( c.getContainerId().getApplicationAttemptId() .getApplicationId()); for (LocalResourceRequest req : e.getValue()) { - XTraceContext.setThreadContext(startCtx); + XTrace.set(startCtx); tracker.handle(new ResourceRequestEvent(req, e.getKey(), ctxt)); - endCtxs = XTraceContext.getThreadContext(endCtxs); + endCtxs.add(XTrace.get()); } } - XTraceContext.joinContext(endCtxs); + for (Context ctx : endCtxs) + XTrace.join(ctx); } private void handleCacheCleanup(LocalizationEvent event) { @@ -406,21 +408,22 @@ private void handleCleanupContainerResources( Container c = rsrcCleanup.getContainer(); Map> rsrcs = rsrcCleanup.getResources(); - Collection start_context = XTraceContext.getThreadContext(); - Collection end_context = new XTraceMetadataCollection(); + Context start_context = XTrace.get(); + Collection end_context = new HashSet(); for (Map.Entry> e : rsrcs.entrySet()) { LocalResourcesTracker tracker = getLocalResourcesTracker(e.getKey(), c.getUser(), c.getContainerId().getApplicationAttemptId() .getApplicationId()); for (LocalResourceRequest req : e.getValue()) { - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); tracker.handle(new ResourceReleaseEvent(req, c.getContainerId())); - end_context = XTraceContext.getThreadContext(end_context); + end_context.add(XTrace.get()); } } - XTraceContext.setThreadContext(end_context); + for (Context ctx : end_context) + XTrace.set(ctx); String locId = ConverterUtils.toString(c.getContainerId()); localizerTracker.cleanupPrivLocalizers(locId); @@ -717,13 +720,14 @@ public void run() { try { // TODO shutdown, better error handling esp. DU while (!Thread.currentThread().isInterrupted()) { - XTraceContext.clearThreadContext(); + XTrace.stop(); try { Future completed = queue.take(); LocalizerResourceRequestEvent assoc = pending.remove(completed); try { Path local = completed.get(); - XTraceContext.joinObject(local); +// // TODO: XTrace deal with this +// XTraceContext.joinObject(local); if (null == assoc) { LOG.error("Localized unkonwn resource to " + completed); // TODO delete @@ -794,7 +798,7 @@ public void addResource(LocalizerResourceRequestEvent request) { */ private LocalResource findNextResource() { // TODO: Synchronization - XTraceContext.clearThreadContext(); + XTrace.stop(); for (Iterator i = pending.iterator(); i.hasNext();) { LocalizerResourceRequestEvent evt = i.next(); @@ -824,7 +828,7 @@ private LocalResource findNextResource() { next.setVisibility(evt.getVisibility()); next.setPattern(evt.getPattern()); scheduled.put(nextRsrc, evt); - XTraceContext.clearThreadContext(); + XTrace.stop(); return next; } else { // Need to release acquired lock @@ -955,7 +959,7 @@ LocalizerHeartbeatResponse update( req, stat.getException().getMessage())); break; } - XTraceContext.clearThreadContext(); + XTrace.stop(); } response.setResourceSpecs(rsrcs); return response; @@ -1063,19 +1067,16 @@ private void writeCredentials(Path nmPrivateCTokensPath) static class CacheCleanup extends Thread { private final Dispatcher dispatcher; - private Collection xtrace_context; public CacheCleanup(Dispatcher dispatcher) { super("CacheCleanup"); this.dispatcher = dispatcher; - this.xtrace_context = XTraceContext.getThreadContext(); } @Override @SuppressWarnings("unchecked") // dispatcher not typed public void run() { - XTraceContext.setThreadContext(xtrace_context); - XTraceContext.logEvent(CacheCleanup.class, "Cache Cleanup Thread", "Cache Cleanup Thread started"); + xtrace.log("Cache Cleanup Thread started"); dispatcher.getEventHandler().handle( new LocalizationEvent(LocalizationEventType.CACHE_CLEANUP)); } 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/loghandler/NonAggregatingLogHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java index 62cce082d0e4..d875268b4820 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -43,9 +42,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; - /** * Log Handler which schedules deletion of log files based on the configured log * retention time. @@ -147,18 +143,15 @@ ScheduledThreadPoolExecutor createScheduledThreadPoolExecutor( class LogDeleterRunnable implements Runnable { private String user; private ApplicationId applicationId; - private Collection xtrace_context; public LogDeleterRunnable(String user, ApplicationId applicationId) { this.user = user; this.applicationId = applicationId; - this.xtrace_context = XTraceContext.getThreadContext(); } @Override @SuppressWarnings("unchecked") public void run() { - XTraceContext.setThreadContext(xtrace_context); List rootLogDirs = NonAggregatingLogHandler.this.dirsHandler.getLogDirs(); Path[] localAppLogDirs = new Path[rootLogDirs.size()]; @@ -174,7 +167,6 @@ public void run() { ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED)); NonAggregatingLogHandler.this.delService.delete(user, null, localAppLogDirs); - XTraceContext.clearThreadContext(); } @Override 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/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java index 8ef9bf92be68..41fdb3cd2bfd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java @@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor; import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -38,19 +37,18 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent; -import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin; +import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import com.google.common.base.Preconditions; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; public class ContainersMonitorImpl extends AbstractService implements ContainersMonitor { - final static Log LOG = LogFactory - .getLog(ContainersMonitorImpl.class); + final static XTrace.Logger xtrace = XTrace.getLogger(ContainersMonitorImpl.class); + final static Log LOG = LogFactory.getLog(ContainersMonitorImpl.class); private long monitoringInterval; private MonitoringThread monitoringThread; @@ -312,17 +310,14 @@ boolean isProcessTreeOverLimit(ResourceCalculatorProcessTree pTree, } private class MonitoringThread extends Thread { - private Collection xtrace_context; public MonitoringThread() { super("Container Monitor"); - this.xtrace_context = XTraceContext.getThreadContext(); } @Override public void run() { - XTraceContext.joinContext(xtrace_context); - XTraceContext.logEvent(MonitoringThread.class, "Container Monitor Thread", "Container Monitor Thread started"); + xtrace.log("Container Monitor Thread started"); while (true) { // Print the processTrees for debugging. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java index ff9c72bcccfb..17c4181bbc61 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java @@ -84,12 +84,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider; import org.apache.hadoop.yarn.server.utils.BuilderUtils; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; @SuppressWarnings("unchecked") @Private public class ApplicationMasterService extends AbstractService implements ApplicationMasterProtocol { + private static final XTrace.Logger xtrace = XTrace.getLogger(ApplicationMasterService.class); private static final Log LOG = LogFactory.getLog(ApplicationMasterService.class); private final AMLivelinessMonitor amLivelinessMonitor; private YarnScheduler rScheduler; @@ -532,7 +533,7 @@ public void registerAppAttempt(ApplicationAttemptId attemptId) { // attemptID get registered response.setResponseId(-1); LOG.info("Registering app attempt : " + attemptId); - XTraceContext.logEvent("ApplicationMasterService", "Registering application attempt", "Attempt ID", attemptId); + xtrace.log("Registering app attempt", "Attempt ID", attemptId); responseMap.put(attemptId, response); rmContext.getNMTokenSecretManager().registerApplicationAttempt(attemptId); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index e92532dc6792..f70d73bb01f7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -43,10 +43,10 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; @@ -94,7 +94,8 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Records; -import edu.berkeley.xtrace.XTraceContext; + +import edu.brown.cs.systems.xtrace.XTrace; /** @@ -105,6 +106,7 @@ public class ClientRMService extends AbstractService implements ApplicationClientProtocol { private static final ArrayList EMPTY_APPS_REPORT = new ArrayList(); + private static final XTrace.Logger xtrace = XTrace.getLogger(ClientRMService.class); private static final Log LOG = LogFactory.getLog(ClientRMService.class); final private AtomicInteger applicationCounter = new AtomicInteger(0); @@ -202,7 +204,7 @@ ApplicationId getNewApplicationId() { .newApplicationId(recordFactory, ResourceManager.clusterTimeStamp, applicationCounter.incrementAndGet()); LOG.info("Allocated new applicationId: " + applicationId.getId()); - XTraceContext.logEvent(ClientRMService.class, "ClientRMService", "New application ID allocated", "Application ID", applicationId.getId()); + xtrace.log("New application ID allocated", "Application ID", applicationId.getId()); return applicationId; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index da9d32cae403..6338e6e323e7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -54,7 +54,7 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** * This class manages the list of applications for the resource manager. @@ -62,6 +62,7 @@ public class RMAppManager implements EventHandler, Recoverable { + private static final XTrace.Logger xtrace = XTrace.getLogger(RMAppManager.class); private static final Log LOG = LogFactory.getLog(RMAppManager.class); private int completedAppsMax = YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS; @@ -243,7 +244,7 @@ protected void submitApplication( ApplicationSubmissionContext submissionContext, long submitTime, boolean isRecovered, String user) throws YarnException { ApplicationId applicationId = submissionContext.getApplicationId(); - XTraceContext.logEvent("RMAppManager", "Submitting application", "Application ID", applicationId.getId()); + xtrace.log("Submitting application", "Application ID", applicationId.getId()); // Validation of the ApplicationSubmissionContext needs to be completed // here. Only those fields that are dependent on RM's configuration are diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index fcd199ea9fbe..284a8eaf9f38 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -67,18 +67,18 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEventType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer; +import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager; -import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.webproxy.AppReportFetcher; @@ -90,7 +90,8 @@ import org.apache.hadoop.yarn.webapp.WebApps.Builder; import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; + +import edu.brown.cs.systems.xtrace.XTrace; /** * The ResourceManager is the main class that is a set of components. @@ -402,7 +403,7 @@ public void run() { SchedulerEvent event; while (!stopped && !Thread.currentThread().isInterrupted()) { - XTraceContext.clearThreadContext(); + XTrace.stop(); try { event = eventQueue.take(); } catch (InterruptedException e) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java index cc2088481dc5..5b814a722b52 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java @@ -24,7 +24,6 @@ import java.security.PrivilegedAction; import java.util.ArrayList; import java.util.List; -import java.util.Collection; import java.util.Map; import org.apache.commons.logging.Log; @@ -35,8 +34,8 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.SecretManager.InvalidToken; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; @@ -64,14 +63,14 @@ import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.XTrace; /** * The launch of the AM itself. */ public class AMLauncher implements Runnable { + private static final XTrace.Logger xtrace = XTrace.getLogger(AMLauncher.class); private static final Log LOG = LogFactory.getLog(AMLauncher.class); private ContainerManagementProtocol containerMgrProxy; @@ -85,8 +84,6 @@ public class AMLauncher implements Runnable { @SuppressWarnings("rawtypes") private final EventHandler handler; - private Collection xtrace_context; - public AMLauncher(RMContext rmContext, RMAppAttempt application, AMLauncherEventType eventType, Configuration conf) { this.application = application; @@ -95,7 +92,6 @@ public AMLauncher(RMContext rmContext, RMAppAttempt application, this.rmContext = rmContext; this.handler = rmContext.getDispatcher().getEventHandler(); this.masterContainer = application.getMasterContainer(); - this.xtrace_context = XTraceContext.getThreadContext(); } private void connect() throws IOException { @@ -111,7 +107,7 @@ private void launch() throws IOException, YarnException { application.getSubmissionContext(); LOG.info("Setting up container " + masterContainer + " for AM " + application.getAppAttemptId()); - XTraceContext.logEvent(AMLauncher.class, "AMLauncher", "Setting up container for application master", + xtrace.log("Setting up container for application master", "Container", application.getMasterContainer(), "Application Master ID", application.getAppAttemptId()); ContainerLaunchContext launchContext = createAMContainerLaunchContext(applicationContext, masterContainerID); @@ -251,14 +247,11 @@ protected Token getAMRMToken() { @SuppressWarnings("unchecked") public void run() { - XTraceContext.setThreadContext(this.xtrace_context); - switch (eventType) { case LAUNCH: try { LOG.info("Launching master " + application.getAppAttemptId()); - XTraceContext.logEvent(AMLauncher.class, "AMLauncher", "Launching application master", - "Application Master ID", application.getAppAttemptId()); + xtrace.log("Launching application master", "Application Master ID", application.getAppAttemptId()); launch(); handler.handle(new RMAppAttemptEvent(application.getAppAttemptId(), RMAppAttemptEventType.LAUNCHED)); @@ -273,8 +266,7 @@ public void run() { case CLEANUP: try { LOG.info("Cleaning master " + application.getAppAttemptId()); - XTraceContext.logEvent(AMLauncher.class, "AMLauncher", "Cleaning application master", - "Application Master ID", application.getAppAttemptId()); + xtrace.log("Cleaning application master", "Application Master ID", application.getAppAttemptId()); cleanup(); } catch(IOException ie) { LOG.info("Error cleaning master ", ie); @@ -292,8 +284,6 @@ public void run() { LOG.warn("Received unknown event-type " + eventType + ". Ignoring."); break; } - - XTraceContext.clearThreadContext(); } private void parseAndThrowException(Throwable t) throws YarnException, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java index 7474f603d9d9..24d2621fa651 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java @@ -21,12 +21,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.event.AbstractEvent; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.Context; public class RMAppEvent extends AbstractEvent{ private final ApplicationId appId; - public XTraceMetadata last_transition; + public Context last_transition; public RMAppEvent(ApplicationId appId, RMAppEventType type) { super(type); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java index 18fdfbd38aeb..eeec1eb4f5f6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java @@ -99,11 +99,12 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.resource.Resources; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; @SuppressWarnings({"unchecked", "rawtypes"}) public class RMAppAttemptImpl implements RMAppAttempt, Recoverable { + private static final XTrace.Logger xtrace = XTrace.getLogger(RMAppAttemptImpl.class); private static final Log LOG = LogFactory.getLog(RMAppAttemptImpl.class); private static final RecordFactory recordFactory = RecordFactoryProvider @@ -789,7 +790,7 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt, BuilderUtils.newResourceRequest( AM_CONTAINER_PRIORITY, ResourceRequest.ANY, appAttempt .getSubmissionContext().getResource(), 1); - XTraceContext.logEvent(RMAppAttemptImpl.class, "RMAppAttemptImpl", "Requesting container for ApplicationMaster"); + xtrace.log("Requesting container for ApplicationMaster"); request.rememberContext(); // SchedulerUtils.validateResourceRequests is not necessary because diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java index e00690a07cfa..33047786575d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java @@ -62,7 +62,8 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; + +import edu.brown.cs.systems.xtrace.XTrace; /** * This class is used to keep track of all the applications/containers @@ -74,6 +75,7 @@ @SuppressWarnings("unchecked") public class RMNodeImpl implements RMNode, EventHandler { + private static final XTrace.Logger xtrace = XTrace.getLogger(RMNodeImpl.class); private static final Log LOG = LogFactory.getLog(RMNodeImpl.class); private static final RecordFactory recordFactory = RecordFactoryProvider @@ -560,7 +562,7 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { for (ContainerStatus remoteContainer : statusEvent.getContainers()) { ContainerId containerId = remoteContainer.getContainerId(); - XTraceContext.clearThreadContext(); + XTrace.stop(); containerId.joinContext(); // Don't bother with containers already scheduled for cleanup, or for @@ -569,8 +571,7 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { if (rmNode.containersToClean.contains(containerId)) { LOG.info("Container " + containerId + " already scheduled for " + "cleanup, no further processing"); - XTraceContext.logEvent(RMNodeImpl.class, "Node cleanup", "Container already scheduled for " + - "cleanup, no further processing", "Container ID", containerId); + xtrace.log("Container already scheduled for cleanup", "Container ID", containerId); continue; } if (rmNode.finishedApplications.contains(containerId @@ -578,8 +579,7 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { LOG.info("Container " + containerId + " belongs to an application that is already killed," + " no further processing"); - XTraceContext.logEvent(RMNodeImpl.class, "Node cleanup", "Container belongs to an application that is already killed" - +", no further processing", "Container ID", containerId); + xtrace.log("Container belongs to an application that is already killed", "Container ID", containerId); continue; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index 7ef18c2694e7..1fb444c9b0fb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -80,7 +80,7 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; @LimitedPrivate("yarn") @Evolving @@ -89,6 +89,7 @@ public class CapacityScheduler implements PreemptableResourceScheduler, CapacitySchedulerContext, Configurable { + private static final XTrace.Logger xtrace = XTrace.getLogger(CapacityScheduler.class); private static final Log LOG = LogFactory.getLog(CapacityScheduler.class); private CSQueue root; @@ -639,13 +640,13 @@ private synchronized void nodeUpdate(RMNode nm) { // Process completed containers for (ContainerStatus completedContainer : completedContainers) { ContainerId containerId = completedContainer.getContainerId(); - XTraceContext.clearThreadContext(); + XTrace.stop(); containerId.joinContext(); LOG.debug("Container FINISHED: " + containerId); - XTraceContext.logEvent(CapacityScheduler.class, "Container Finished", "Container Finished", "Container ID",containerId); + xtrace.log("Container Finished", "Container ID",containerId); completedContainer(getRMContainer(containerId), completedContainer, RMContainerEventType.FINISHED); - XTraceContext.clearThreadContext(); + XTrace.stop(); } // Now node data structures are upto date and ready for scheduling. @@ -697,7 +698,7 @@ private synchronized void nodeUpdate(RMNode nm) { } private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { - XTraceContext.clearThreadContext(); + XTrace.stop(); containerId.joinContext(); // Get the application for the finished container @@ -713,7 +714,7 @@ private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode } application.containerLaunchedOnNode(containerId, node.getNodeID()); - XTraceContext.clearThreadContext(); + XTrace.stop(); } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index 8f6c0b7ad95d..48c8c4756eb2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -67,7 +67,7 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; @Private @Unstable @@ -1304,7 +1304,7 @@ private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode nod FiCaSchedulerApp application, Priority priority, ResourceRequest request, NodeType type, RMContainer rmContainer) { - XTraceContext.clearThreadContext(); + XTrace.stop(); request.joinContext(); try { // xtrace try @@ -1392,7 +1392,7 @@ private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode nod } } finally { // xtrace finally - XTraceContext.clearThreadContext(); + XTrace.stop(); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index d29c6eeaa29f..97b0b8e1c9e8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -60,14 +60,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; -import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; +import org.apache.hadoop.yarn.util.resource.Resources; import com.google.common.collect.HashMultiset; import com.google.common.collect.Multiset; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** * Represents an Application from the viewpoint of the scheduler. @@ -311,13 +311,13 @@ synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node, synchronized public List pullNewlyAllocatedContainers() { List returnContainerList = new ArrayList( newlyAllocatedContainers.size()); - Collection start_context = XTraceContext.getThreadContext(); + Context start_context = XTrace.get(); for (RMContainer rmContainer : newlyAllocatedContainers) { rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(), RMContainerEventType.ACQUIRED)); returnContainerList.add(rmContainer.getContainer()); rmContainer.getContainerId().rememberContext(); - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); } newlyAllocatedContainers.clear(); return returnContainerList; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java index 6929e904c78c..ad6e4b9aba82 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java @@ -35,10 +35,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -51,6 +49,7 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; @@ -74,7 +73,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent; @@ -92,7 +90,7 @@ import com.google.common.annotations.VisibleForTesting; -import edu.berkeley.xtrace.XTraceContext; +import edu.brown.cs.systems.xtrace.XTrace; /** * A scheduler that schedules resources between a set of queues. The scheduler @@ -126,6 +124,7 @@ public class FairScheduler implements ResourceScheduler { private QueueManager queueMgr; private Clock clock; + private static final XTrace.Logger xtrace = XTrace.getLogger(); private static final Log LOG = LogFactory.getLog(FairScheduler.class); private static final ResourceCalculator RESOURCE_CALCULATOR = @@ -855,7 +854,7 @@ public Allocation allocate(ApplicationAttemptId appAttemptId, * Process a container which has launched on a node, as reported by the node. */ private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) { - XTraceContext.clearThreadContext(); + XTrace.stop(); containerId.joinContext(); // Get the application for the finished container @@ -869,7 +868,7 @@ private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode no } application.containerLaunchedOnNode(containerId, node.getNodeID()); - XTraceContext.clearThreadContext(); + XTrace.stop(); } /** @@ -897,13 +896,13 @@ private synchronized void nodeUpdate(RMNode nm) { // Process completed containers for (ContainerStatus completedContainer : completedContainers) { ContainerId containerId = completedContainer.getContainerId(); - XTraceContext.clearThreadContext(); + XTrace.stop(); containerId.joinContext(); LOG.debug("Container FINISHED: " + containerId); - XTraceContext.logEvent(CapacityScheduler.class, "Container Finished", "Container Finished", "Container ID", containerId); + xtrace.log("Container Finished", "Container ID", containerId); completedContainer(getRMContainer(containerId), completedContainer, RMContainerEventType.FINISHED); - XTraceContext.clearThreadContext(); + XTrace.stop(); } // Assign new containers... diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index 4cc55b9240b9..c65afb572336 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -37,12 +36,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; 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.Token; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.QueueACL; @@ -51,7 +48,9 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; @@ -75,7 +74,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerUtils; @@ -92,15 +90,15 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import edu.berkeley.xtrace.XTraceContext; -import edu.berkeley.xtrace.XTraceMetadata; -import edu.berkeley.xtrace.XTraceMetadataCollection; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; @LimitedPrivate("yarn") @Evolving @SuppressWarnings("unchecked") public class FifoScheduler implements ResourceScheduler, Configurable { + private static final XTrace.Logger xtrace = XTrace.getLogger(FifoScheduler.class); private static final Log LOG = LogFactory.getLog(FifoScheduler.class); private static final RecordFactory recordFactory = @@ -264,9 +262,9 @@ public Allocation allocate( clusterResource, minimumAllocation, maximumAllocation); // Release containers - Collection start_context = XTraceContext.getThreadContext(); + Context start_context = XTrace.get(); for (ContainerId releasedContainer : release) { - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); releasedContainer.joinContext(); RMContainer rmContainer = getRMContainer(releasedContainer); if (rmContainer == null) { @@ -282,7 +280,7 @@ public Allocation allocate( SchedulerUtils.RELEASED_CONTAINER), RMContainerEventType.RELEASED); } - XTraceContext.setThreadContext(start_context); + XTrace.set(start_context); synchronized (application) { @@ -563,7 +561,7 @@ private int assignOffSwitchContainers(FiCaSchedulerNode node, private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, int assignableContainers, ResourceRequest request, NodeType type) { - XTraceContext.clearThreadContext(); + XTrace.stop(); request.joinContext(); LOG.debug("assignContainers:" + @@ -620,7 +618,7 @@ private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application } } - XTraceContext.clearThreadContext(); + XTrace.stop(); return assignedContainers; } @@ -643,13 +641,13 @@ private synchronized void nodeUpdate(RMNode rmNode) { // Process completed containers for (ContainerStatus completedContainer : completedContainers) { ContainerId containerId = completedContainer.getContainerId(); - XTraceContext.clearThreadContext(); + XTrace.stop(); containerId.joinContext(); LOG.debug("Container FINISHED: " + containerId); - XTraceContext.logEvent(CapacityScheduler.class, "Container Finished", "Container Finished", "Container ID", containerId); + xtrace.log("Container Finished", "Container ID", containerId); containerCompleted(getRMContainer(containerId), completedContainer, RMContainerEventType.FINISHED); - XTraceContext.clearThreadContext(); + XTrace.stop(); } if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource, @@ -727,7 +725,7 @@ public void handle(SchedulerEvent event) { } private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { - XTraceContext.clearThreadContext(); + XTrace.stop(); containerId.joinContext(); // Get the application for the finished container @@ -745,7 +743,7 @@ private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode } application.containerLaunchedOnNode(containerId, node.getNodeID()); - XTraceContext.clearThreadContext(); + XTrace.stop(); } @Lock(FifoScheduler.class) From 8d969afe27f9432f727c86a6fa5578985cb77c24 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 24 Mar 2014 18:00:33 -0700 Subject: [PATCH 057/112] Small fix --- .../hadoop/hdfs/protocol/datatransfer/PacketHeader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java index fe4dfd0b77b7..e7aadeb8fbc2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java @@ -135,8 +135,8 @@ public boolean hasXTraceContext() { } public void joinXTraceContext() { - ByteString xbs = proto.getXtrace(); - XTrace.join(xbs.toByteArray()); + if (proto.hasXtrace()) + XTrace.join(proto.getXtrace().toByteArray()); } @Override From e61df715c8a430684ef891fe1a091583c7cdc81d Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 24 Mar 2014 21:33:57 -0700 Subject: [PATCH 058/112] Peer cache is long lived, don't attribute to first task we see --- .../src/main/java/org/apache/hadoop/hdfs/PeerCache.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java index 424b641c8c39..ce9dbeace978 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java @@ -131,6 +131,7 @@ private synchronized void startExpiryDaemon() { daemon = new Daemon(new Runnable() { @Override public void run() { + XTrace.stop(); // Long-lived, don't attribute to a task try { PeerCache.this.run(); } catch(InterruptedException e) { From 577dc100514430db57b6ebcb53a3836b18ed632f Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 25 Mar 2014 08:57:59 -0700 Subject: [PATCH 059/112] Whoops... didn't commit this properly... --- .../main/java/org/apache/hadoop/hdfs/PeerCache.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java index ce9dbeace978..9cd9cebd6c07 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java @@ -22,17 +22,20 @@ import java.util.List; import java.util.Map.Entry; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.LinkedListMultimap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Time; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.LinkedListMultimap; + +import edu.brown.cs.systems.xtrace.XTrace; + /** * A cache of input stream sockets to Data Node. */ From 9c25021a6acd2686feb1f6f653f329e65c42111d Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 26 Mar 2014 13:11:22 -0700 Subject: [PATCH 060/112] Add a few log messages to NativeIO, why not --- .../main/java/org/apache/hadoop/io/nativeio/NativeIO.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java index 5e9fe5be3223..87c3297de039 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java @@ -26,6 +26,8 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +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; @@ -34,8 +36,7 @@ import org.apache.hadoop.util.NativeCodeLoader; import org.apache.hadoop.util.Shell; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import edu.brown.cs.systems.xtrace.XTrace; /** * JNI wrappers for various native IO-related calls not available in Java. @@ -90,6 +91,7 @@ public static class POSIX { write. */ public static final int SYNC_FILE_RANGE_WAIT_AFTER = 4; + private static final XTrace.Logger xtrace = XTrace.getLogger(NativeIO.class); private static final Log LOG = LogFactory.getLog(NativeIO.class); private static boolean nativeLoaded = false; @@ -184,6 +186,7 @@ public static void posixFadviseIfPossible( if (nativeLoaded && fadvisePossible) { try { posix_fadvise(fd, offset, len, flags); + xtrace.log("posix_fadvise", "offset", offset, "len", len, "flags", flags); } catch (UnsupportedOperationException uoe) { fadvisePossible = false; } catch (UnsatisfiedLinkError ule) { @@ -205,6 +208,7 @@ public static void syncFileRangeIfPossible( if (nativeLoaded && syncFileRangePossible) { try { sync_file_range(fd, offset, nbytes, flags); + xtrace.log("sync_file_range", "offset", offset, "nbytes", nbytes, "flags", flags); } catch (UnsupportedOperationException uoe) { syncFileRangePossible = false; } catch (UnsatisfiedLinkError ule) { From 4f30e5342de27970ee13f3c2d8a6cec95dd74fe1 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 26 Mar 2014 14:40:01 -0700 Subject: [PATCH 061/112] Instrument call queue in IPC Server --- .../java/org/apache/hadoop/ipc/Server.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index c03b15cbb834..fdad2e8c328c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -117,6 +117,7 @@ import com.google.protobuf.Message.Builder; import edu.brown.cs.systems.resourcetracing.resources.Network; +import edu.brown.cs.systems.resourcetracing.resources.QueueResource; import edu.brown.cs.systems.xtrace.Context; import edu.brown.cs.systems.xtrace.XTrace; @@ -375,6 +376,8 @@ public static boolean isRpcInvocation() { private final boolean tcpNoDelay; // if T then disable Nagle's Algorithm volatile private boolean running = true; // true while server runs + + private QueueResource callQueueInstrumentation; // xresourcetracing instrumentation for the callqueue. could be better but will do for now private BlockingQueue callQueue; // queued calls private List connectionList = @@ -484,7 +487,9 @@ public static class Call { private ByteBuffer rpcResponse; // the response for this call private final RPC.RpcKind rpcKind; private final byte[] clientId; + private Context start_context; // the X-Trace context this was received with + public long enqueue, dequeue, complete; // Timers for queue instrumentation; hacky but quick private Context response_context; // the X-Trace context before sending the response public Call(int id, int retryCount, Writable param, @@ -1891,6 +1896,8 @@ private void processRpcRequest(RpcRequestHeaderProto header, Call call = new Call(header.getCallId(), header.getRetryCount(), rpcRequest, this, ProtoUtil.convert(header.getRpcKind()), header .getClientId().toByteArray()); + callQueueInstrumentation.enqueue(); + call.enqueue = System.nanoTime(); callQueue.put(call); // queue the call; maybe blocked here incRpcCount(); // Increment the rpc count } @@ -2036,7 +2043,12 @@ public void run() { XTrace.stop(); try { final Call call = callQueue.take(); // pop the queue; maybe blocked here + call.dequeue = System.nanoTime(); XTrace.set(call.start_context); + callQueueInstrumentation.starting(call.enqueue, call.dequeue); + + try { // xtrace try + if (LOG.isDebugEnabled()) { LOG.debug(getName() + ": " + call + " for RpcKind " + call.rpcKind); } @@ -2118,6 +2130,11 @@ public Writable run() throws Exception { } responder.doRespond(call); } + + } finally { // xtrace finally + call.complete = System.nanoTime(); + callQueueInstrumentation.finished(call.enqueue, call.dequeue, call.complete); + } } catch (InterruptedException e) { if (running) { // unexpected -- log it LOG.info(getName() + " unexpectedly interrupted", e); @@ -2196,6 +2213,7 @@ protected Server(String bindAddress, int port, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT); } this.callQueue = new LinkedBlockingQueue(maxQueueSize); + this.callQueueInstrumentation = new QueueResource("Server-"+System.identityHashCode(this)+"-callQueue", handlerCount); this.maxIdleTime = 2 * conf.getInt( CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT); From 630659b4571c9e835b564aa2993f5172fb49b872 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 1 Apr 2014 11:45:39 -0700 Subject: [PATCH 062/112] Slight changes to the RPC server and client response sending/processing threads, to make sure the correct XTrace metadata is set at all times --- .../java/org/apache/hadoop/ipc/Client.java | 27 ++++++++++++++----- .../java/org/apache/hadoop/ipc/Server.java | 9 +++++++ 2 files changed, 29 insertions(+), 7 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index 74f447be187c..a8da2a582850 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -95,6 +95,7 @@ import com.google.protobuf.ByteString; import com.google.protobuf.CodedOutputStream; +import edu.brown.cs.systems.resourcetracing.resources.Network; import edu.brown.cs.systems.xtrace.Context; import edu.brown.cs.systems.xtrace.XTrace; @@ -243,6 +244,9 @@ void checkResponse(RpcResponseHeaderProto header) throws IOException { } } } + if (header.hasXtrace()) { + XTrace.set(header.getXtrace().toByteArray()); + } } Call createCall(RPC.RpcKind rpcKind, Writable rpcRequest) { @@ -1007,14 +1011,24 @@ private void receiveRpcResponse() { } touch(); + XTrace.stop(); try { - int totalLen = in.readInt(); - RpcResponseHeaderProto header = - RpcResponseHeaderProto.parseDelimitedFrom(in); + Network.ignore(true); + int totalLen; + RpcResponseHeaderProto header; + try { + totalLen = in.readInt(); + header = RpcResponseHeaderProto.parseDelimitedFrom(in); + } finally { + Network.ignore(false); + } checkResponse(header); int headerLen = header.getSerializedSize(); headerLen += CodedOutputStream.computeRawVarint32Size(headerLen); + + Network.Read.alreadyStarted(in); + Network.Read.alreadyFinished(in, headerLen); int callId = header.getCallId(); if (LOG.isDebugEnabled()) @@ -1022,14 +1036,11 @@ private void receiveRpcResponse() { Call call = calls.get(callId); RpcStatusProto status = header.getStatus(); - if (header.hasXtrace()) { - ByteString xbs = header.getXtrace(); - call.xtrace = Context.parse(xbs.toByteArray()); - } if (status == RpcStatusProto.SUCCESS) { Writable value = ReflectionUtils.newInstance(valueClass, conf); value.readFields(in); // read value + call.xtrace = XTrace.get(); call.setRpcResponse(value); calls.remove(callId); @@ -1074,6 +1085,8 @@ private void receiveRpcResponse() { } } catch (IOException e) { markClosed(e); + } finally { + XTrace.stop(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index fdad2e8c328c..ebf0e5640fc1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -875,6 +875,7 @@ private class Responder extends Thread { @Override public void run() { LOG.info(getName() + ": starting"); + XTrace.stop(); SERVER.set(Server.this); try { doRunLoop(); @@ -1003,6 +1004,10 @@ private boolean processResponse(LinkedList responseQueue, boolean done = false; // there is more data for this channel. int numElements = 0; Call call = null; + + if (!inHandler) + XTrace.stop(); + try { synchronized (responseQueue) { // @@ -1030,6 +1035,7 @@ private boolean processResponse(LinkedList responseQueue, return true; } if (!call.rpcResponse.hasRemaining()) { + xtrace.log("Finished writing RPC response"); //Clear out the response buffer so it can be collected call.rpcResponse = null; call.connection.decRpcCount(); @@ -1043,6 +1049,7 @@ private boolean processResponse(LinkedList responseQueue, + " Wrote " + numBytes + " bytes."); } } else { + xtrace.log("Wrote partial RPC response, enqueueing for later finish"); // // If we were unable to write the entire response out, then // insert in Selector queue. @@ -1079,6 +1086,8 @@ private boolean processResponse(LinkedList responseQueue, done = true; // error. no more data for this channel. closeConnection(call.connection); } + if (!inHandler) + XTrace.stop(); } return done; } From 7fd7a39ab383465266c7a717fff235d811462641 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 1 Apr 2014 14:39:08 -0700 Subject: [PATCH 063/112] Add ability to have connection-per-client in a single process --- .../org/apache/hadoop/ipc/ProtobufRpcEngine.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java index 0b66a05f373d..c43fe5883610 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java @@ -112,6 +112,7 @@ public ProtocolProxy getProtocolMetaInfoProxy( private static class Invoker implements RpcInvocationHandler { private final Map returnTypes = new ConcurrentHashMap(); + private boolean isClientCached = true; private boolean isClosed = false; private final Client.ConnectionId remoteId; private final Client client; @@ -132,7 +133,11 @@ private Invoker(Class protocol, InetSocketAddress addr, private Invoker(Class protocol, Client.ConnectionId connId, Configuration conf, SocketFactory factory) { this.remoteId = connId; - this.client = CLIENTS.getClient(conf, factory, RpcResponseWrapper.class); + this.isClientCached = conf.getBoolean("xtrace.client.cached", true); + if (this.isClientCached) + this.client = CLIENTS.getClient(conf, factory, RpcResponseWrapper.class); + else + this.client = new Client(RpcResponseWrapper.class, conf, factory); this.protocolName = RPC.getProtocolName(protocol); this.clientProtocolVersion = RPC .getProtocolVersion(protocol); @@ -267,7 +272,10 @@ public Object invoke(Object proxy, Method method, Object[] args) public void close() throws IOException { if (!isClosed) { isClosed = true; - CLIENTS.stopClient(client); + if (isClientCached) + CLIENTS.stopClient(client); + else + client.stop(); } } From 5c48be6f9f1408abcf1fc551b3c7808b9a2f29c6 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Sun, 20 Apr 2014 19:34:43 -0700 Subject: [PATCH 064/112] For now, comment out the random sleep if a complete call fails after writing a file, because it produces way too much arbitrary interference --- .../org/apache/hadoop/hdfs/DFSOutputStream.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index a6f3a1388fd9..9a550b4aea28 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -1968,13 +1968,14 @@ private void completeFile(ExtendedBlock last) throws IOException { DFSClient.LOG.info(msg); throw new IOException(msg); } - try { - Thread.sleep(400); - if (Time.now() - localstart > 5000) { - DFSClient.LOG.info("Could not complete " + src + " retrying..."); - } - } catch (InterruptedException ie) { - } + // Jon: comment this out for now because it's random, arbitrary interference +// try { +// Thread.sleep(400); +// if (Time.now() - localstart > 5000) { +// DFSClient.LOG.info("Could not complete " + src + " retrying..."); +// } +// } catch (InterruptedException ie) { +// } } } } From 9d9fe905220dc761a897e8066595015ed62b457a Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 21 Apr 2014 08:40:37 -0700 Subject: [PATCH 065/112] Add 5ms sleep instead of 400ms sleep --- .../org/apache/hadoop/hdfs/DFSOutputStream.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 9a550b4aea28..b9a7d753acfd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -1968,14 +1968,14 @@ private void completeFile(ExtendedBlock last) throws IOException { DFSClient.LOG.info(msg); throw new IOException(msg); } - // Jon: comment this out for now because it's random, arbitrary interference -// try { -// Thread.sleep(400); -// if (Time.now() - localstart > 5000) { -// DFSClient.LOG.info("Could not complete " + src + " retrying..."); -// } -// } catch (InterruptedException ie) { -// } + // Jon: reduce this from 400 to 5 just so as not to taint our exps + try { + Thread.sleep(5); + if (Time.now() - localstart > 5000) { + DFSClient.LOG.info("Could not complete " + src + " retrying..."); + } + } catch (InterruptedException ie) { + } } } } From 055a6db91a26dee713f70fa590e07fb0483139d4 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 23 Apr 2014 12:06:40 -0700 Subject: [PATCH 066/112] Add instrumentation of DN heartbeats --- .../hadoop/hdfs/server/datanode/BPServiceActor.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 447ea2e7d5f4..bb6587b20523 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -58,6 +58,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Maps; +import edu.brown.cs.systems.resourcetracing.backgroundtasks.HDFSBackgroundTask; +import edu.brown.cs.systems.resourcetracing.backgroundtasks.HeartbeatBackgroundTask; import edu.brown.cs.systems.xtrace.XTrace; /** @@ -526,7 +528,11 @@ private void offerService() throws Exception { // lastHeartbeat = startTime; if (!dn.areHeartbeatsDisabledForTests()) { + HDFSBackgroundTask.HEARTBEAT.start(); + long begin = System.nanoTime(); HeartbeatResponse resp = sendHeartBeat(); + HDFSBackgroundTask.HEARTBEAT.end(System.nanoTime() - begin); + assert resp != null; dn.getMetrics().addHeartbeat(now() - startTime); @@ -539,6 +545,8 @@ private void offerService() throws Exception { bpos.updateActorStatesFromHeartbeat( this, resp.getNameNodeHaState()); + HDFSBackgroundTask.DN_HEARTBEAT_PROCESS.start(); + begin = System.nanoTime(); long startProcessCommands = now(); if (!processCommand(resp.getCommands())) continue; @@ -548,6 +556,7 @@ private void offerService() throws Exception { + "ms to process " + resp.getCommands().length + " commands from NN"); } + HDFSBackgroundTask.DN_HEARTBEAT_PROCESS.end(System.nanoTime() - begin); } } if (pendingReceivedRequests > 0 From d1a31b79e8fb1e5131401294c6fab11e247690c7 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 23 Apr 2014 12:24:07 -0700 Subject: [PATCH 067/112] Bad import removed --- .../org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index bb6587b20523..305992b5c798 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -59,7 +59,6 @@ import com.google.common.collect.Maps; import edu.brown.cs.systems.resourcetracing.backgroundtasks.HDFSBackgroundTask; -import edu.brown.cs.systems.resourcetracing.backgroundtasks.HeartbeatBackgroundTask; import edu.brown.cs.systems.xtrace.XTrace; /** From 3f7f7a8e9c39176e61362b0b8b0f2207ac26f260 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 23 Apr 2014 14:15:07 -0700 Subject: [PATCH 068/112] Instrument more background tasks --- .../hadoop/hdfs/server/datanode/BPServiceActor.java | 10 ++++++---- .../hdfs/server/datanode/BlockPoolSliceStorage.java | 10 ++++++++++ .../apache/hadoop/hdfs/server/datanode/DataNode.java | 10 ++++++++++ .../hadoop/hdfs/server/datanode/DataStorage.java | 9 +++++++++ .../fsdataset/impl/FsDatasetAsyncDiskService.java | 11 ++++++++--- .../server/datanode/fsdataset/impl/FsDatasetImpl.java | 8 +++++++- 6 files changed, 50 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 305992b5c798..ee5e398a44e5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -529,8 +529,12 @@ private void offerService() throws Exception { if (!dn.areHeartbeatsDisabledForTests()) { HDFSBackgroundTask.HEARTBEAT.start(); long begin = System.nanoTime(); - HeartbeatResponse resp = sendHeartBeat(); - HDFSBackgroundTask.HEARTBEAT.end(System.nanoTime() - begin); + HeartbeatResponse resp; + try { + resp = sendHeartBeat(); + } finally { + HDFSBackgroundTask.HEARTBEAT.end(System.nanoTime() - begin); + } assert resp != null; dn.getMetrics().addHeartbeat(now() - startTime); @@ -544,7 +548,6 @@ private void offerService() throws Exception { bpos.updateActorStatesFromHeartbeat( this, resp.getNameNodeHaState()); - HDFSBackgroundTask.DN_HEARTBEAT_PROCESS.start(); begin = System.nanoTime(); long startProcessCommands = now(); if (!processCommand(resp.getCommands())) @@ -555,7 +558,6 @@ private void offerService() throws Exception { + "ms to process " + resp.getCommands().length + " commands from NN"); } - HDFSBackgroundTask.DN_HEARTBEAT_PROCESS.end(System.nanoTime() - begin); } } if (pendingReceivedRequests > 0 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/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 4a58e4258167..11d49cae73ee 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,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.BlockingService; +import edu.brown.cs.systems.resourcetracing.backgroundtasks.HDFSBackgroundTask; import edu.brown.cs.systems.xtrace.XTrace; /********************************************************** @@ -1547,6 +1548,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; @@ -1638,6 +1642,8 @@ public void run() { IOUtils.closeStream(out); IOUtils.closeStream(in); IOUtils.closeSocket(sock); + + HDFSBackgroundTask.REPLICATION.end(System.nanoTime() - begin); } } } @@ -1948,11 +1954,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/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."); From 411f677ddc2d6520bcab8028ef3dd41a14725ada Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 24 Apr 2014 14:30:43 -0700 Subject: [PATCH 069/112] Add a hacky addition to allow kinda throttling of background block replication --- .../blockmanagement/DatanodeDescriptor.java | 43 +++++++++- .../blockmanagement/DatanodeManager.java | 7 +- .../hdfs/server/namenode/FSNamesystem.java | 85 +++++++++++++++++-- 3 files changed, 122 insertions(+), 13 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java index 56ac12f5ffe3..1f375fc8cd84 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java @@ -23,6 +23,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Queue; +import java.util.Random; import java.util.Set; import java.util.TreeSet; @@ -71,7 +72,7 @@ private static class BlockQueue { synchronized boolean offer(E e) { return blockq.offer(e); } - + /** Dequeue */ synchronized List poll(int numBlocks) { if (numBlocks <= 0 || blockq.isEmpty()) { @@ -404,6 +405,46 @@ int getNumberOfBlocksToBeInvalidated() { public List getReplicationCommand(int maxTransfers) { return replicateBlocks.poll(maxTransfers); } + + private static final Random r = new Random(); + + public List getReplicationCommand(int maxTransfers, long maxBytes) { + long blocksRemainingforReplication = 0; + long totalBytes = 0; + long totalBlocks = 0; + List results = null; + synchronized(replicateBlocks) { // Jon: terrible but quick + blocksRemainingforReplication = replicateBlocks.blockq.size(); + while (totalBytes < maxBytes && totalBlocks < maxTransfers) { + // Decide whether we're allowed the next block. Shouldn't be probabilitic, but easiest for now + BlockTargetPair next = replicateBlocks.blockq.peek(); + if (next==null) + break; + long nextsize = next.block.getNumBytes(); + if (totalBytes + nextsize > maxBytes) { + double probability = (maxBytes - totalBytes) / (double) nextsize; + if (r.nextDouble() > probability) + break; + } + + // Get the next block + next = replicateBlocks.blockq.poll(); + if (next==null) // shouldn't happen, but just in case + break; + + // Add the block + if (results==null) + results = new ArrayList(); + results.add(next); + + // Increment counters + totalBytes+=next.block.getNumBytes(); + totalBlocks++; + } + } + System.out.println("Replication: toReplicate="+blocksRemainingforReplication + " maxBlocks="+maxTransfers + " maxBytes="+maxBytes + " blocks="+totalBlocks+" bytes="+totalBytes); + return results; + } public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) { List blocks = recoverBlocks.poll(maxTransfers); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index 90541351dc50..cf48156023ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -1147,11 +1147,12 @@ private void setDatanodeDead(DatanodeDescriptor node) { node.setLastUpdate(0); } - /** Handle heartbeat from datanodes. */ + /** Handle heartbeat from datanodes. + * @param maxBytes */ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, final String blockPoolId, long capacity, long dfsUsed, long remaining, long blockPoolUsed, - int xceiverCount, int maxTransfers, int failedVolumes + int xceiverCount, int maxTransfers, long maxBytes, int failedVolumes ) throws IOException { synchronized (heartbeatManager) { synchronized (datanodeMap) { @@ -1223,7 +1224,7 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, final List cmds = new ArrayList(); //check pending replication List pendingList = nodeinfo.getReplicationCommand( - maxTransfers); + maxTransfers, maxBytes); if (pendingList != null) { cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId, pendingList)); 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 a3f3fcf6ced5..5bbec8208cee 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,10 @@ 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; + /*************************************************** @@ -3916,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 void setReplicationTotalBytesPerSecond(long bytes) { + String command = "set:"+bytes; + PubSub.publish(replication_topic, StringMessage.newBuilder().setMessage(command).build()); + } + + public 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: @@ -3936,9 +4000,12 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg, try { final int maxTransfer = blockManager.getMaxReplicationStreams() - xmitsInProgress; + long maxBytes = Long.MAX_VALUE; + if ((System.currentTimeMillis()-last_replication_command) Date: Thu, 24 Apr 2014 14:32:35 -0700 Subject: [PATCH 070/112] Oh, and make methods static --- .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 5bbec8208cee..41d8e985e76d 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 @@ -3933,12 +3933,12 @@ String getRegistrationID() { * X-Trace: sends a pubsub message specifying the bytes per second allowed by replication * @param bytes */ - public void setReplicationTotalBytesPerSecond(long bytes) { + public static void setReplicationTotalBytesPerSecond(long bytes) { String command = "set:"+bytes; PubSub.publish(replication_topic, StringMessage.newBuilder().setMessage(command).build()); } - public void clearReplication() { + public static void clearReplication() { String command = "clear"; PubSub.publish(replication_topic, StringMessage.newBuilder().setMessage(command).build()); } From 99afc5b2629e9f4ff5381f32a8323ad94dd8a6b5 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Sat, 26 Apr 2014 21:05:10 -0700 Subject: [PATCH 071/112] Fix divide by zero exception --- .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 41d8e985e76d..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 @@ -4001,8 +4001,10 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg, final int maxTransfer = blockManager.getMaxReplicationStreams() - xmitsInProgress; long maxBytes = Long.MAX_VALUE; - if ((System.currentTimeMillis()-last_replication_command) 0) { + maxBytes = replication_bps / numLiveNodes; + } DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat( nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed, xceiverCount, maxTransfer, maxBytes, failedVolumes); From bd81bf518ef8213692ae92c39cf07f04256289ea Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Sun, 27 Apr 2014 14:52:48 -0700 Subject: [PATCH 072/112] Try alternative approach to replication throttling using the balancer bandwidth --- .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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 db6d2c0809cb..c82173e17073 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 @@ -4002,8 +4002,13 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg, - xmitsInProgress; long maxBytes = Long.MAX_VALUE; long numLiveNodes = blockManager.getDatanodeManager().getNumLiveDataNodes(); +// if ((System.currentTimeMillis()-last_replication_command) 0) { +// maxBytes = replication_bps / numLiveNodes; +// } + // use setBalancerBandwidth to do throttling for us if ((System.currentTimeMillis()-last_replication_command) 0) { - maxBytes = replication_bps / numLiveNodes; + long balancer_bandwidth = replication_bps / numLiveNodes; + blockManager.getDatanodeManager().setBalancerBandwidth(balancer_bandwidth); } DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat( nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed, From 6f1ccc4546e378be741ce660f354cb04adf96b65 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Sun, 27 Apr 2014 14:55:33 -0700 Subject: [PATCH 073/112] Also default to large balancer bandwidth --- .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 2 ++ 1 file changed, 2 insertions(+) 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 c82173e17073..1eb648fb5814 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 @@ -4009,6 +4009,8 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg, if ((System.currentTimeMillis()-last_replication_command) 0) { long balancer_bandwidth = replication_bps / numLiveNodes; blockManager.getDatanodeManager().setBalancerBandwidth(balancer_bandwidth); + } else { + blockManager.getDatanodeManager().setBalancerBandwidth(Integer.MAX_VALUE); } DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat( nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed, From 9e22b3c99b0d62eea63b019e68d74a0d4f2ee071 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Sun, 27 Apr 2014 22:45:36 -0700 Subject: [PATCH 074/112] Use both approaches simultaneously! --- .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 1eb648fb5814..074a98cfa9fd 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 @@ -4002,10 +4002,10 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg, - xmitsInProgress; long maxBytes = Long.MAX_VALUE; long numLiveNodes = blockManager.getDatanodeManager().getNumLiveDataNodes(); -// if ((System.currentTimeMillis()-last_replication_command) 0) { -// maxBytes = replication_bps / numLiveNodes; -// } - // use setBalancerBandwidth to do throttling for us + if ((System.currentTimeMillis()-last_replication_command) 0) { + maxBytes = replication_bps / numLiveNodes; + } + // use setBalancerBandwidth to do smooth throttling too if ((System.currentTimeMillis()-last_replication_command) 0) { long balancer_bandwidth = replication_bps / numLiveNodes; blockManager.getDatanodeManager().setBalancerBandwidth(balancer_bandwidth); From f0ee65f2e0f6aea1d36c75466c54610da11e90e2 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 30 Apr 2014 16:33:22 -0700 Subject: [PATCH 075/112] Removed balancer bandwidth - unnecessary --- .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 7 ------- 1 file changed, 7 deletions(-) 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 074a98cfa9fd..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 @@ -4005,13 +4005,6 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg, if ((System.currentTimeMillis()-last_replication_command) 0) { maxBytes = replication_bps / numLiveNodes; } - // use setBalancerBandwidth to do smooth throttling too - if ((System.currentTimeMillis()-last_replication_command) 0) { - long balancer_bandwidth = replication_bps / numLiveNodes; - blockManager.getDatanodeManager().setBalancerBandwidth(balancer_bandwidth); - } else { - blockManager.getDatanodeManager().setBalancerBandwidth(Integer.MAX_VALUE); - } DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat( nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed, xceiverCount, maxTransfer, maxBytes, failedVolumes); From b4d6e3e49eb075ba48b2d5796bc76b78b4ae70a7 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 5 Jun 2014 11:38:40 -0400 Subject: [PATCH 076/112] Moved the AspectJ stuff to the root pom. I'm really not sure which one it's supposed to go in though. --- hadoop-project-dist/pom.xml | 42 ----------------------------------- pom.xml | 44 +++++++++++++++++++++++++++++++++++++ 2 files changed, 44 insertions(+), 42 deletions(-) diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml index 2f5915be1e85..788cbdd9644c 100644 --- a/hadoop-project-dist/pom.xml +++ b/hadoop-project-dist/pom.xml @@ -48,52 +48,10 @@ hadoop-annotations provided - - - org.aspectj - aspectjrt - 1.7.3 - - - edu.brown.cs.systems - xtrace - 3.0 - - - edu.brown.cs.systems - xtrace-resource-tracing - 1.0 - - - org.codehaus.mojo - aspectj-maven-plugin - 1.5 - - - - edu.brown.cs.systems - xtrace-resource-tracing - - - - - - - compile - - - 1.6 - 1.6 - 1.6 - synchronization - - - - org.apache.maven.plugins maven-jar-plugin diff --git a/pom.xml b/pom.xml index 6dfd5d1f4c16..e34f03ef39c0 100644 --- a/pom.xml +++ b/pom.xml @@ -23,6 +23,24 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs Apache Hadoop Main pom + + + org.aspectj + aspectjrt + 1.7.3 + + + edu.brown.cs.systems + xtrace + 3.0 + + + edu.brown.cs.systems + xtrace-resource-tracing + 1.0 + + + @@ -161,6 +179,32 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs + + org.codehaus.mojo + aspectj-maven-plugin + 1.5 + + + + edu.brown.cs.systems + xtrace-resource-tracing + + + + + + + compile + + + 1.6 + 1.6 + 1.6 + synchronization + + + + org.apache.maven.plugins maven-enforcer-plugin From b36e2f5fa8142798b88389398e3c9e6215011b5f Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 5 Jun 2014 11:44:30 -0400 Subject: [PATCH 077/112] Indentation in pom --- pom.xml | 84 ++++++++++++++++++++++++++++----------------------------- 1 file changed, 42 insertions(+), 42 deletions(-) diff --git a/pom.xml b/pom.xml index e34f03ef39c0..aa53d238a9ac 100644 --- a/pom.xml +++ b/pom.xml @@ -23,22 +23,22 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs Apache Hadoop Main pom - - - org.aspectj - aspectjrt - 1.7.3 - - - edu.brown.cs.systems - xtrace - 3.0 - - - edu.brown.cs.systems - xtrace-resource-tracing - 1.0 - + + + org.aspectj + aspectjrt + 1.7.3 + + + edu.brown.cs.systems + xtrace + 3.0 + + + edu.brown.cs.systems + xtrace-resource-tracing + 1.0 + @@ -179,32 +179,32 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs - - org.codehaus.mojo - aspectj-maven-plugin - 1.5 - - - - edu.brown.cs.systems - xtrace-resource-tracing - - - - - - - compile - - - 1.6 - 1.6 - 1.6 - synchronization - - - - + + org.codehaus.mojo + aspectj-maven-plugin + 1.5 + + + + edu.brown.cs.systems + xtrace-resource-tracing + + + + + + + compile + + + 1.6 + 1.6 + 1.6 + synchronization + + + + org.apache.maven.plugins maven-enforcer-plugin From 8cd00ffd4a7de8f3f820ce9e95017c815d05f135 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 25 Jul 2014 15:12:46 -0400 Subject: [PATCH 078/112] Add showWeaveInfo=true to pom --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index aa53d238a9ac..13598682fa57 100644 --- a/pom.xml +++ b/pom.xml @@ -201,6 +201,8 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 1.6 1.6 synchronization + true + true From 7332cde2fd7fa3dba69894032f04ec8b86944db8 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 13 Aug 2014 11:27:52 -0400 Subject: [PATCH 079/112] Add throttling points to HDFS --- .../src/main/java/org/apache/hadoop/ipc/Server.java | 4 +++- hadoop-common-project/hadoop-nfs/pom.xml | 2 +- .../hadoop/hdfs/server/datanode/BlockReceiver.java | 6 ++++++ .../hadoop/hdfs/server/datanode/BlockSender.java | 6 ++++++ pom.xml | 10 ++++++++++ 5 files changed, 26 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index ebf0e5640fc1..3ce0d0a84c84 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -116,6 +116,7 @@ import com.google.protobuf.Message; import com.google.protobuf.Message.Builder; +import edu.brown.cs.systems.resourcethrottling.LocalThrottlingPoints; import edu.brown.cs.systems.resourcetracing.resources.Network; import edu.brown.cs.systems.resourcetracing.resources.QueueResource; import edu.brown.cs.systems.xtrace.Context; @@ -2221,7 +2222,8 @@ protected Server(String bindAddress, int port, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT); } - this.callQueue = new LinkedBlockingQueue(maxQueueSize); + this.callQueue = LocalThrottlingPoints.getThrottlingQueue("Server-"+System.identityHashCode(this)+"-callQueue"); +// this.callQueue = new LinkedBlockingQueue(maxQueueSize); this.callQueueInstrumentation = new QueueResource("Server-"+System.identityHashCode(this)+"-callQueue", handlerCount); this.maxIdleTime = 2 * conf.getInt( CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, diff --git a/hadoop-common-project/hadoop-nfs/pom.xml b/hadoop-common-project/hadoop-nfs/pom.xml index 324c9830890d..c01962818b2b 100644 --- a/hadoop-common-project/hadoop-nfs/pom.xml +++ b/hadoop-common-project/hadoop-nfs/pom.xml @@ -92,7 +92,7 @@ com.google.guava guava - 11.0.2 + 17.0 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 c47e46fa291c..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,6 +53,8 @@ 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; @@ -62,6 +64,7 @@ **/ 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; @@ -629,6 +632,9 @@ private int receivePacket() throws IOException { throttler.throttle(len); } + // Retro throttle + throttlingpoint.throttle(); + return lastPacketInBlock?-1:len; } 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 c1c67cd8934b..b92ae0400056 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,8 @@ 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; /** @@ -91,6 +93,7 @@ */ 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 = @@ -554,6 +557,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; } diff --git a/pom.xml b/pom.xml index 13598682fa57..be669e538673 100644 --- a/pom.xml +++ b/pom.xml @@ -39,6 +39,16 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs xtrace-resource-tracing 1.0 + + edu.brown.cs.systems + resourcethrottling + 1.0 + + + com.google.guava + guava + 17.0 + From ab3d98ef5ddfd5ecb902fe745318816c928ef66b Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Fri, 15 Aug 2014 13:03:03 -0400 Subject: [PATCH 080/112] Rename the throttling point on the RPC server --- .../src/main/java/org/apache/hadoop/ipc/Server.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index 3ce0d0a84c84..b8d9d840ee17 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -120,6 +120,7 @@ import edu.brown.cs.systems.resourcetracing.resources.Network; import edu.brown.cs.systems.resourcetracing.resources.QueueResource; import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.Reporter.Utils; import edu.brown.cs.systems.xtrace.XTrace; /** An abstract IPC service. IPC calls take a single {@link Writable} as a @@ -2222,7 +2223,7 @@ protected Server(String bindAddress, int port, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT); } - this.callQueue = LocalThrottlingPoints.getThrottlingQueue("Server-"+System.identityHashCode(this)+"-callQueue"); + this.callQueue = LocalThrottlingPoints.getThrottlingQueue(Utils.getProcessName()+"-RPCServer"); // this.callQueue = new LinkedBlockingQueue(maxQueueSize); this.callQueueInstrumentation = new QueueResource("Server-"+System.identityHashCode(this)+"-callQueue", handlerCount); this.maxIdleTime = 2 * conf.getInt( From 2478bebb20652869f0f4cb5ceb2377404c3f5ade Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 1 Sep 2014 13:06:55 -0400 Subject: [PATCH 081/112] Put servername in the call queue to allow multiple servers --- .../src/main/java/org/apache/hadoop/ipc/Server.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index b8d9d840ee17..764949ebb859 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -2223,7 +2223,7 @@ protected Server(String bindAddress, int port, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT); } - this.callQueue = LocalThrottlingPoints.getThrottlingQueue(Utils.getProcessName()+"-RPCServer"); + this.callQueue = LocalThrottlingPoints.getThrottlingQueue(Utils.getProcessName()+"-"+serverName); // this.callQueue = new LinkedBlockingQueue(maxQueueSize); this.callQueueInstrumentation = new QueueResource("Server-"+System.identityHashCode(this)+"-callQueue", handlerCount); this.maxIdleTime = 2 * conf.getInt( From 3a983461646cb33b4cec470dab4b5e3c983f3913 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 1 Sep 2014 15:49:16 -0400 Subject: [PATCH 082/112] Fix up the propagation of X-Trace metadata from NodeManager to Containers (specifically, YarnChild and MRAppMaster containers --- .../src/main/java/org/apache/hadoop/util/Shell.java | 3 +++ .../src/main/java/org/apache/hadoop/mapred/YarnChild.java | 2 ++ .../java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java | 2 ++ 3 files changed, 7 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java index e826fbc91b96..3cd4b10cd021 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java @@ -32,6 +32,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import edu.brown.cs.systems.xtrace.XTrace; + /** * A base class for running a Unix command. * @@ -384,6 +386,7 @@ private void runCommand() throws IOException { if (environment != null) { builder.environment().putAll(this.environment); } + builder.environment().putAll(XTrace.environment()); // // TODO: implement this in new xtrace // // put xtrace context if there is one, merging as appropriate 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 c0bbdcdbd3e2..cef40aa91a92 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 @@ -75,6 +75,8 @@ class YarnChild { static volatile TaskAttemptID taskid = null; public static void main(String[] args) throws Throwable { + // Load the XTrace context from the parent process + XTrace.set(System.getenv()); xtrace.log("YarnChild starting"); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); LOG.debug("Child starting"); 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 f9ca2024780a..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 @@ -1293,6 +1293,8 @@ 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 = From 3750df394a2425c4d30ef21a1ccf0f2fb5ad3870 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 1 Sep 2014 15:49:47 -0400 Subject: [PATCH 083/112] Let the examples in the examples jar start X-Trace tasks, useful for debugging and verifying that everything is working --- .../src/main/java/org/apache/hadoop/util/ProgramDriver.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java index e52dacccbb4f..956fb58b673e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java @@ -26,6 +26,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import edu.brown.cs.systems.xtrace.XTrace; + /** A driver that is used to run programs added to it */ @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @@ -135,6 +137,10 @@ public int run(String[] args) printUsage(programs); return -1; } + + XTrace.startTask(true); + XTrace.setTenantClass(55); + XTrace.getLogger("ProgramDriver").log("Executing example program", "ProgramName", args[0]); // Remove the leading argument and call main String[] new_args = new String[args.length - 1]; From fcdbe7fb9f61331352f4b38ce10c4064f1496ba2 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 1 Sep 2014 15:51:07 -0400 Subject: [PATCH 084/112] Base64 encoding isn't quite compatible with HTTP headers because it uses a disallowed character '=' for padding. For now just switch to Base16, which is less efficient but only includes alphanumeric characters. Could tweak X-Trace base64 to not include padding but don't want to risk violating any existing instrumentation, no time to fix it --- .../java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 f5d2d856330f..fe4ceb725f7a 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 @@ -294,7 +294,7 @@ 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.base64(); + String xtrace_string = XTrace.base16(); if (xtrace_string!=null) connection.addRequestProperty("X-Trace", xtrace_string); // set the read timeout @@ -317,7 +317,7 @@ protected void copyFromHost(MapHost host) throws IOException { if (rc != HttpURLConnection.HTTP_OK) { String xtrace_context = connection.getHeaderField("X-Trace"); if (xtrace_context!=null) { - XTrace.join(Context.parse(xtrace_context)); + XTrace.join(Context.parseBase16(xtrace_context)); } xtrace.log("Got invalid response code " + rc + " from host", "URL", url, "Message", connection.getResponseMessage()); throw new IOException( From 050428e9197350621471b0208ee9a9122401d41b Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 1 Sep 2014 16:27:57 -0400 Subject: [PATCH 085/112] Fixed up the inclusion of xtrace metadata in the shuffle header --- .../hadoop/mapreduce/task/reduce/Fetcher.java | 2 - .../mapreduce/task/reduce/ShuffleHeader.java | 43 +------------------ .../apache/hadoop/mapred/ShuffleHandler.java | 1 - 3 files changed, 2 insertions(+), 44 deletions(-) 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 fe4ceb725f7a..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 @@ -428,8 +428,6 @@ private TaskAttemptID[] copyMapOutput(MapHost host, try { ShuffleHeader header = new ShuffleHeader(); header.readFields(input); - XTrace.stop(); - header.joinContext(); mapId = TaskAttemptID.forName(header.mapId); compressedLength = header.compressedLength; decompressedLength = header.uncompressedLength; 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 527efa369180..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 @@ -72,14 +72,7 @@ public void readFields(DataInput in) throws IOException { compressedLength = WritableUtils.readVLong(in); uncompressedLength = WritableUtils.readVLong(in); forReduce = WritableUtils.readVInt(in); - long taskid = WritableUtils.readVLong(in); - long opid = WritableUtils.readVLong(in); - if (taskid!=0L) { - XTraceMetadata.Builder builder = XTraceMetadata.newBuilder().setTaskID(taskid); - if (opid!=0L) - builder.addParentEventID(opid); - m = Context.parse(builder.build().toByteArray()); - } + XTrace.join(WritableUtils.readCompressedByteArray(in)); } public void write(DataOutput out) throws IOException { @@ -87,39 +80,7 @@ public void write(DataOutput out) throws IOException { WritableUtils.writeVLong(out, compressedLength); WritableUtils.writeVLong(out, uncompressedLength); WritableUtils.writeVInt(out, forReduce); - WritableUtils.writeVLong(out, getXTraceTaskID()); - WritableUtils.writeVLong(out, getXTraceOpID()); + WritableUtils.writeCompressedByteArray(out, XTrace.bytes()); } - private long getXTraceTaskID() { - if (m!=null) { - try { - return XTraceMetadata.parseFrom(m.bytes()).getTaskID(); - } catch (Exception e) { - } - } - return 0L; - } - - private long getXTraceOpID() { - if (m!=null) { - try { - XTraceMetadata xmd = XTraceMetadata.parseFrom(m.bytes()); - if (xmd.getParentEventIDCount() > 0) - return xmd.getParentEventID(0); - } catch (Exception e) { - } - } - return 0L; - } - - - public void rememberContext() { - m = XTrace.get(); - } - - public void joinContext() { -// // TODO: what even is this -// XTraceContext.joinChildProcess(m); - } } \ No newline at end of file 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 515d15d6bca5..5a9b5b83e6b3 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 @@ -609,7 +609,6 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, xtrace.log("Sending map output", "Reduce", reduce, "Map ID", mapId, "Job ID", jobID); final ShuffleHeader header = new ShuffleHeader(mapId, info.partLength, info.rawLength, reduce); - header.rememberContext(); final DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); From 26367308f10c1c8493b2a5f8e5e7d0ac1cb227a4 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 3 Sep 2014 16:00:09 -0400 Subject: [PATCH 086/112] Put throttling queue only on NN server for now --- .../src/main/java/org/apache/hadoop/ipc/Server.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index 764949ebb859..3a40e7261f9f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -2223,8 +2223,10 @@ protected Server(String bindAddress, int port, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT); } - this.callQueue = LocalThrottlingPoints.getThrottlingQueue(Utils.getProcessName()+"-"+serverName); -// this.callQueue = new LinkedBlockingQueue(maxQueueSize); + if ("NameNode".equals(Utils.getProcessName())) // Hack; put a throttling queue on NN only for now + this.callQueue = LocalThrottlingPoints.getThrottlingQueue(Utils.getProcessName()+"-"+serverName); + else + this.callQueue = new LinkedBlockingQueue(maxQueueSize); this.callQueueInstrumentation = new QueueResource("Server-"+System.identityHashCode(this)+"-callQueue", handlerCount); this.maxIdleTime = 2 * conf.getInt( CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, From fa384e1a4127ebbc59fa5e6c1c35ca3071730a65 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 4 Sep 2014 17:12:16 -0400 Subject: [PATCH 087/112] Only add call queue instrumentation for namenode for now --- .../java/org/apache/hadoop/ipc/Server.java | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index 3a40e7261f9f..2126f9bd589a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -379,7 +379,7 @@ public static boolean isRpcInvocation() { volatile private boolean running = true; // true while server runs - private QueueResource callQueueInstrumentation; // xresourcetracing instrumentation for the callqueue. could be better but will do for now + private QueueResource callQueueInstrumentation = null; // xresourcetracing instrumentation for the callqueue. could be better but will do for now private BlockingQueue callQueue; // queued calls private List connectionList = @@ -1907,7 +1907,8 @@ private void processRpcRequest(RpcRequestHeaderProto header, Call call = new Call(header.getCallId(), header.getRetryCount(), rpcRequest, this, ProtoUtil.convert(header.getRpcKind()), header .getClientId().toByteArray()); - callQueueInstrumentation.enqueue(); + if (callQueueInstrumentation!=null) + callQueueInstrumentation.enqueue(); call.enqueue = System.nanoTime(); callQueue.put(call); // queue the call; maybe blocked here incRpcCount(); // Increment the rpc count @@ -2056,7 +2057,8 @@ public void run() { final Call call = callQueue.take(); // pop the queue; maybe blocked here call.dequeue = System.nanoTime(); XTrace.set(call.start_context); - callQueueInstrumentation.starting(call.enqueue, call.dequeue); + if (callQueueInstrumentation!=null) + callQueueInstrumentation.starting(call.enqueue, call.dequeue); try { // xtrace try @@ -2144,7 +2146,8 @@ public Writable run() throws Exception { } finally { // xtrace finally call.complete = System.nanoTime(); - callQueueInstrumentation.finished(call.enqueue, call.dequeue, call.complete); + if (callQueueInstrumentation!=null) + callQueueInstrumentation.finished(call.enqueue, call.dequeue, call.complete); } } catch (InterruptedException e) { if (running) { // unexpected -- log it @@ -2223,11 +2226,13 @@ protected Server(String bindAddress, int port, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT); } - if ("NameNode".equals(Utils.getProcessName())) // Hack; put a throttling queue on NN only for now + if ("NameNode".equals(Utils.getProcessName())) { + // Hack; put a throttling queue on NN only for now this.callQueue = LocalThrottlingPoints.getThrottlingQueue(Utils.getProcessName()+"-"+serverName); - else - this.callQueue = new LinkedBlockingQueue(maxQueueSize); - this.callQueueInstrumentation = new QueueResource("Server-"+System.identityHashCode(this)+"-callQueue", handlerCount); + this.callQueueInstrumentation = new QueueResource("Server-"+System.identityHashCode(this)+"-callQueue", handlerCount); + } else { + this.callQueue = new LinkedBlockingQueue(maxQueueSize); + } this.maxIdleTime = 2 * conf.getInt( CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT); From 18f9ecb5ee24f3f92fa1918ca48c65fa5b03c2fc Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 15 Sep 2014 16:04:37 -0400 Subject: [PATCH 088/112] Foolishly forgot to use base 16 decoding of XTrace header in shuffle handler --- .../src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 5a9b5b83e6b3..a676be7a5c22 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 @@ -452,7 +452,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) String xtrace_context = request.getHeader("X-Trace"); if (xtrace_context!=null) - XTrace.set(xtrace_context); + XTrace.set(xtrace_context, XTrace.ENCODING.BASE16); final Map> q = new QueryStringDecoder(request.getUri()).getParameters(); From 5f9ec89461e54e7fe5c20aa2206447ee68754256 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 15 Sep 2014 18:00:40 -0400 Subject: [PATCH 089/112] Add throttling point to ShuffleHandler and to RawFileSystem --- .../org/apache/hadoop/fs/RawLocalFileSystem.java | 13 ++++++++++++- .../org/apache/hadoop/mapred/ShuffleHandler.java | 4 ++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java index e27a22735875..4c591185ee4d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java @@ -42,6 +42,9 @@ import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.StringUtils; +import edu.brown.cs.systems.resourcethrottling.LocalThrottlingPoints; +import edu.brown.cs.systems.resourcethrottling.ThrottlingPoint; + /**************************************************************** * Implement the FileSystem API for the raw local filesystem. * @@ -49,6 +52,9 @@ @InterfaceAudience.Public @InterfaceStability.Stable public class RawLocalFileSystem extends FileSystem { + private static ThrottlingPoint writer_throttlingpoint = LocalThrottlingPoints.getThrottlingPoint("LocalDiskOut"); + private static ThrottlingPoint reader_throttlingpoint = LocalThrottlingPoints.getThrottlingPoint("LocalDiskIn"); + static final URI NAME = URI.create("file:///"); private Path workingDir; @@ -123,7 +129,7 @@ class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor private long position; public LocalFSFileInputStream(Path f) throws IOException { - this.fis = new TrackingFileInputStream(pathToFile(f)); + this.fis = new FileInputStream(pathToFile(f)); } @Override @@ -154,6 +160,7 @@ public boolean seekToNewSource(long targetPos) throws IOException { @Override public int read() throws IOException { + reader_throttlingpoint.throttle(); try { int value = fis.read(); if (value >= 0) { @@ -167,6 +174,7 @@ public int read() throws IOException { @Override public int read(byte[] b, int off, int len) throws IOException { + reader_throttlingpoint.throttle(); try { int value = fis.read(b, off, len); if (value > 0) { @@ -181,6 +189,7 @@ public int read(byte[] b, int off, int len) throws IOException { @Override public int read(long position, byte[] b, int off, int len) throws IOException { + reader_throttlingpoint.throttle(); ByteBuffer bb = ByteBuffer.wrap(b, off, len); try { return fis.getChannel().read(bb, position); @@ -232,6 +241,7 @@ private LocalFSFileOutputStream(Path f, boolean append) throws IOException { public void flush() throws IOException { fos.flush(); } @Override public void write(byte[] b, int off, int len) throws IOException { + writer_throttlingpoint.throttle(); try { fos.write(b, off, len); } catch (IOException e) { // unexpected exception @@ -241,6 +251,7 @@ public void write(byte[] b, int off, int len) throws IOException { @Override public void write(int b) throws IOException { + writer_throttlingpoint.throttle(); try { fos.write(b); } catch (IOException e) { // unexpected exception 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 a676be7a5c22..849b22e47b02 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 @@ -113,6 +113,8 @@ 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.xtrace.Context; import edu.brown.cs.systems.xtrace.XTrace; @@ -120,6 +122,7 @@ 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; @@ -580,6 +583,7 @@ 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 From bd2fce13ee0ece842f715ac23c956a29bd22b5dc Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 16 Sep 2014 11:08:24 -0400 Subject: [PATCH 090/112] Add some special handling for MR shuffle handler network output --- .../hadoop/mapred/FadvisedChunkedFile.java | 16 +++++++++++++++- .../apache/hadoop/mapred/FadvisedFileRegion.java | 16 +++++++++++++++- .../org/apache/hadoop/mapred/ShuffleHandler.java | 9 ++++++++- 3 files changed, 38 insertions(+), 3 deletions(-) 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 849b22e47b02..63d132ebd711 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 @@ -115,6 +115,7 @@ 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; @@ -580,10 +581,12 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, } } - protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, + protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, final Channel ch, String user, String jobId, String mapId, int reduce) throws IOException { shuffle_throttler.throttle(); + final Network netrsrc = Network.Write(((InetSocketAddress) ch.getRemoteAddress()).getAddress()); + // 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 @@ -629,7 +632,9 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, final FadvisedFileRegion partition = new FadvisedFileRegion(spill, info.startOffset, info.partLength, manageOsCache, readaheadLength, readaheadPool, spillfile.getAbsolutePath()); + netrsrc.starting(ch, null); writeFuture = ch.write(partition); + netrsrc.finished(ch, info.partLength, null); writeFuture.addListener(new ChannelFutureListener() { // TODO error handling; distinguish IO/connection failures, // attribute to appropriate spill output @@ -644,7 +649,9 @@ public void operationComplete(ChannelFuture future) { info.startOffset, info.partLength, sslFileBufferSize, manageOsCache, readaheadLength, readaheadPool, spillfile.getAbsolutePath()); + netrsrc.starting(ch, null); writeFuture = ch.write(chunk); + netrsrc.finished(ch, info.partLength, null); } metrics.shuffleConnections.incr(); metrics.shuffleOutputBytes.incr(info.partLength); // optimistic From 28076ab347849c560e6c20b548b64bbbf6a970cc Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 16 Sep 2014 12:17:08 -0400 Subject: [PATCH 091/112] Remove previous commit instrumentation of shuffle handler, was too much --- .../main/java/org/apache/hadoop/mapred/ShuffleHandler.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) 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 63d132ebd711..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 @@ -581,11 +581,10 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, } } - protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, final Channel ch, + protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, String user, String jobId, String mapId, int reduce) throws IOException { shuffle_throttler.throttle(); - final Network netrsrc = Network.Write(((InetSocketAddress) ch.getRemoteAddress()).getAddress()); // TODO replace w/ rsrc alloc // $x/$user/appcache/$appId/output/$mapId @@ -632,9 +631,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, final Channel c final FadvisedFileRegion partition = new FadvisedFileRegion(spill, info.startOffset, info.partLength, manageOsCache, readaheadLength, readaheadPool, spillfile.getAbsolutePath()); - netrsrc.starting(ch, null); writeFuture = ch.write(partition); - netrsrc.finished(ch, info.partLength, null); writeFuture.addListener(new ChannelFutureListener() { // TODO error handling; distinguish IO/connection failures, // attribute to appropriate spill output @@ -649,9 +646,7 @@ public void operationComplete(ChannelFuture future) { info.startOffset, info.partLength, sslFileBufferSize, manageOsCache, readaheadLength, readaheadPool, spillfile.getAbsolutePath()); - netrsrc.starting(ch, null); writeFuture = ch.write(chunk); - netrsrc.finished(ch, info.partLength, null); } metrics.shuffleConnections.incr(); metrics.shuffleOutputBytes.incr(info.partLength); // optimistic From 0cbaa3780e193722682a095a4395874c73fee964 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 17 Sep 2014 12:55:51 -0400 Subject: [PATCH 092/112] Remove throttling point from local file system, put in spill thread --- .../main/java/org/apache/hadoop/fs/RawLocalFileSystem.java | 7 ------- .../src/main/java/org/apache/hadoop/mapred/YarnChild.java | 1 + .../src/main/java/org/apache/hadoop/mapred/MapTask.java | 5 +++++ 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java index 4c591185ee4d..f9ac0089d2f5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java @@ -52,8 +52,6 @@ @InterfaceAudience.Public @InterfaceStability.Stable public class RawLocalFileSystem extends FileSystem { - private static ThrottlingPoint writer_throttlingpoint = LocalThrottlingPoints.getThrottlingPoint("LocalDiskOut"); - private static ThrottlingPoint reader_throttlingpoint = LocalThrottlingPoints.getThrottlingPoint("LocalDiskIn"); static final URI NAME = URI.create("file:///"); private Path workingDir; @@ -160,7 +158,6 @@ public boolean seekToNewSource(long targetPos) throws IOException { @Override public int read() throws IOException { - reader_throttlingpoint.throttle(); try { int value = fis.read(); if (value >= 0) { @@ -174,7 +171,6 @@ public int read() throws IOException { @Override public int read(byte[] b, int off, int len) throws IOException { - reader_throttlingpoint.throttle(); try { int value = fis.read(b, off, len); if (value > 0) { @@ -189,7 +185,6 @@ public int read(byte[] b, int off, int len) throws IOException { @Override public int read(long position, byte[] b, int off, int len) throws IOException { - reader_throttlingpoint.throttle(); ByteBuffer bb = ByteBuffer.wrap(b, off, len); try { return fis.getChannel().read(bb, position); @@ -241,7 +236,6 @@ private LocalFSFileOutputStream(Path f, boolean append) throws IOException { public void flush() throws IOException { fos.flush(); } @Override public void write(byte[] b, int off, int len) throws IOException { - writer_throttlingpoint.throttle(); try { fos.write(b, off, len); } catch (IOException e) { // unexpected exception @@ -251,7 +245,6 @@ public void write(byte[] b, int off, int len) throws IOException { @Override public void write(int b) throws IOException { - writer_throttlingpoint.throttle(); try { fos.write(b); } catch (IOException e) { // unexpected exception 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 cef40aa91a92..14d341c78fee 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 @@ -215,6 +215,7 @@ public Object run() throws Exception { } } finally { xtrace.log("YarnChild exiting"); + Thread.sleep(1000); // TODO: xtrace join parent process // XTraceContext.joinParentProcess(); RPC.stopProxy(umbilical); 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 fe9dd8c3ce67..3db7d084a373 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 @@ -75,6 +75,8 @@ import org.apache.hadoop.util.StringInterner; 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.XTrace; /** A Map task. */ @@ -92,6 +94,8 @@ public class MapTask extends Task { private static final XTrace.Logger xtrace = XTrace.getLogger(MapTask.class); private static final Log LOG = LogFactory.getLog(MapTask.class.getName()); + private static ThrottlingPoint spill_throttler = LocalThrottlingPoints.getThrottlingPoint("MRSpill"); + private Progress mapPhase; private Progress sortPhase; @@ -1618,6 +1622,7 @@ 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); + spill_throttler.throttle(); //approximate the length of the output file to be the length of the //buffer + header lengths for the partitions From 46948718cab697ff7fb4e9b8ea151b7ebf5e635c Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 17 Sep 2014 13:09:09 -0400 Subject: [PATCH 093/112] Add some cpu tracking... test --- .../src/main/java/org/apache/hadoop/mapreduce/Mapper.java | 3 +++ .../src/main/java/org/apache/hadoop/mapreduce/Reducer.java | 3 +++ 2 files changed, 6 insertions(+) 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(); From 0acf0bc76e25d04c991e7dc4642a8cb1f376df4b Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 18 Sep 2014 13:17:17 -0400 Subject: [PATCH 094/112] Put throttling points in new position --- .../org/apache/hadoop/fs/RawLocalFileSystem.java | 3 --- .../main/java/org/apache/hadoop/mapred/IFile.java | 15 +++++++++++++-- .../java/org/apache/hadoop/mapred/MapTask.java | 4 ---- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java index f9ac0089d2f5..32cfc0aac081 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java @@ -42,9 +42,6 @@ import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.StringUtils; -import edu.brown.cs.systems.resourcethrottling.LocalThrottlingPoints; -import edu.brown.cs.systems.resourcethrottling.ThrottlingPoint; - /**************************************************************** * Implement the FileSystem API for the raw local filesystem. * 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..b77077a170c2 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,9 @@ 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 writer_throttler = LocalThrottlingPoints.getThrottlingPoint("IFileWriter"); + private static ThrottlingPoint reader_throttler = LocalThrottlingPoints.getThrottlingPoint("IFileReader"); /** * IFile.Writer to write out intermediate map-outputs. @@ -228,6 +233,8 @@ public void append(K key, V value) throws IOException { WritableUtils.getVIntSize(keyLength) + WritableUtils.getVIntSize(valueLength); ++numRecordsWritten; + + writer_throttler.throttle(); } public void append(DataInputBuffer key, DataInputBuffer value) @@ -254,6 +261,8 @@ public void append(DataInputBuffer key, DataInputBuffer value) WritableUtils.getVIntSize(keyLength) + WritableUtils.getVIntSize(valueLength); ++numRecordsWritten; + + writer_throttler.throttle(); } // Required for mark/reset @@ -378,6 +387,8 @@ public long getPosition() throws IOException { * @throws IOException */ private int readData(byte[] buf, int off, int len) throws IOException { + reader_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/MapTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java index 3db7d084a373..77b6b26eae50 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 @@ -75,8 +75,6 @@ import org.apache.hadoop.util.StringInterner; 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.XTrace; /** A Map task. */ @@ -94,8 +92,6 @@ public class MapTask extends Task { private static final XTrace.Logger xtrace = XTrace.getLogger(MapTask.class); private static final Log LOG = LogFactory.getLog(MapTask.class.getName()); - private static ThrottlingPoint spill_throttler = LocalThrottlingPoints.getThrottlingPoint("MRSpill"); - private Progress mapPhase; private Progress sortPhase; From e8d08a81232f7ece048bbcaba989ae7cda611310 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 18 Sep 2014 13:30:42 -0400 Subject: [PATCH 095/112] remove spill throttler --- .../src/main/java/org/apache/hadoop/mapred/MapTask.java | 1 - 1 file changed, 1 deletion(-) 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 77b6b26eae50..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 @@ -1618,7 +1618,6 @@ 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); - spill_throttler.throttle(); //approximate the length of the output file to be the length of the //buffer + header lengths for the partitions From a95b2edb42b77edd1e0dd486771deb2aaebc4451 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Thu, 18 Sep 2014 21:05:53 -0400 Subject: [PATCH 096/112] Share throttling point for ifile --- .../src/main/java/org/apache/hadoop/mapred/IFile.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) 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 b77077a170c2..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 @@ -61,8 +61,7 @@ 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 writer_throttler = LocalThrottlingPoints.getThrottlingPoint("IFileWriter"); - private static ThrottlingPoint reader_throttler = LocalThrottlingPoints.getThrottlingPoint("IFileReader"); + private static ThrottlingPoint throttler = LocalThrottlingPoints.getThrottlingPoint("IFile"); /** * IFile.Writer to write out intermediate map-outputs. @@ -234,7 +233,7 @@ public void append(K key, V value) throws IOException { WritableUtils.getVIntSize(valueLength); ++numRecordsWritten; - writer_throttler.throttle(); + throttler.throttle(); } public void append(DataInputBuffer key, DataInputBuffer value) @@ -262,7 +261,7 @@ public void append(DataInputBuffer key, DataInputBuffer value) WritableUtils.getVIntSize(valueLength); ++numRecordsWritten; - writer_throttler.throttle(); + throttler.throttle(); } // Required for mark/reset @@ -387,7 +386,7 @@ public long getPosition() throws IOException { * @throws IOException */ private int readData(byte[] buf, int off, int len) throws IOException { - reader_throttler.throttle(); + throttler.throttle(); int bytesRead = 0; while (bytesRead < len) { From 2ad2121ca69798d8daf7aed49e94baf4926ef69a Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Sat, 20 Sep 2014 19:57:15 -0400 Subject: [PATCH 097/112] Do this manually... --- .../src/main/java/org/apache/hadoop/mapred/YarnChild.java | 4 ++++ 1 file changed, 4 insertions(+) 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 14d341c78fee..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,6 +62,7 @@ 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; /** @@ -75,6 +76,9 @@ class YarnChild { 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"); From d61ef05698158d99090a5da623dbed1ba4e0a9d4 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 21 Oct 2014 19:11:35 -0400 Subject: [PATCH 098/112] Manually disable datanode hostname check. Later versions of HDFS make this configurable. --- .../hdfs/server/blockmanagement/DatanodeManager.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index cf48156023ab..ff93d73d5385 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -1138,9 +1138,10 @@ public List getDatanodeListForReport( * @return boolean true if name resolution successful or address is loopback */ private static boolean isNameResolved(InetAddress address) { - String hostname = address.getHostName(); - String ip = address.getHostAddress(); - return !hostname.equals(ip) || address.isLoopbackAddress(); +// String hostname = address.getHostName(); +// String ip = address.getHostAddress(); +// return !hostname.equals(ip) || address.isLoopbackAddress(); + return true; } private void setDatanodeDead(DatanodeDescriptor node) { From 4fbbbbcea2cde93252e6e2b4bdd75713ecbbf88b Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 10 Dec 2014 11:42:11 -0500 Subject: [PATCH 099/112] Add HDFS config option to specify whether to fadvise long files --- .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java | 2 ++ .../org/apache/hadoop/hdfs/server/datanode/BlockSender.java | 4 +++- .../java/org/apache/hadoop/hdfs/server/datanode/DNConf.java | 4 ++++ 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 0bc1c25ee8ed..5cae58c8e7db 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -93,6 +93,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final boolean DFS_DATANODE_SYNC_BEHIND_WRITES_DEFAULT = false; public static final String DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY = "dfs.datanode.drop.cache.behind.reads"; public static final boolean DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT = false; + public static final String DFS_DATANODE_FADVISE_SEQUENTIAL = "dfs.datanode.fadvise.sequential.reads"; + public static final boolean DFS_DATANODE_FADVISE_SEQUENTIAL_DEFAULT = true; public static final String DFS_DATANODE_USE_DN_HOSTNAME = "dfs.datanode.use.datanode.hostname"; public static final boolean DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT = false; 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 b92ae0400056..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 @@ -148,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 @@ -183,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) { @@ -655,7 +657,7 @@ long sendBlock(DataOutputStream out, OutputStream baseStream, 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); 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); From 220170e62c349f8bd4863ed3add03f35cdc3cb33 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 10 Dec 2014 13:07:09 -0500 Subject: [PATCH 100/112] Add some logging for datanode selection --- .../apache/hadoop/hdfs/DFSInputStream.java | 4 ++++ .../blockmanagement/DatanodeManager.java | 22 +++++++++++++++++-- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index c025ee8d46bd..3ea21ecb5095 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -816,6 +816,10 @@ private DNAddrPair chooseDataNode(LocatedBlock block) throws IOException { while (true) { DatanodeInfo[] nodes = block.getLocations(); + String s = ""; + for (DatanodeInfo info : nodes) + s = s + " " + info.toString(); + System.out.printf("%d chooseDataNode: %s\n", Thread.currentThread().getId(), s); try { DatanodeInfo chosenNode = bestNode(nodes, deadNodes); final String dnAddr = diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index ff93d73d5385..b2c03f973653 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -329,15 +329,25 @@ public void sortLocatedBlocks(final String targethost, //sort the blocks // As it is possible for the separation of node manager and datanode, // here we should get node but not datanode only . + StringBuilder sortOutput = new StringBuilder(); + sortOutput.append("sortLocatedBlocks status:\n"); + Node client = getDatanodeByHost(targethost); if (client == null) { + sortOutput.append(" - client==null, targethost="+targethost+"\n"); List hosts = new ArrayList (1); hosts.add(targethost); String rName = dnsToSwitchMapping.resolve(hosts).get(0); - if (rName != null) + if (rName != null) { + sortOutput.append(" - rname = " + rName+"\n"); + sortOutput.append(" - create NodeBase: " + rName + NodeBase.PATH_SEPARATOR_STR + targethost + "\n"); client = new NodeBase(rName + NodeBase.PATH_SEPARATOR_STR + targethost); + } else { + sortOutput.append(" - rname == null " + rName+"\n"); + } } - + + sortOutput.append(" - avoidStaleDataNodesForRead=="+avoidStaleDataNodesForRead+"\n"); Comparator comparator = avoidStaleDataNodesForRead ? new DFSUtil.DecomStaleComparator(staleInterval) : DFSUtil.DECOM_COMPARATOR; @@ -346,7 +356,15 @@ public void sortLocatedBlocks(final String targethost, networktopology.pseudoSortByDistance(client, b.getLocations()); // Move decommissioned/stale datanodes to the bottom Arrays.sort(b.getLocations(), comparator); + + sortOutput.append(" - Sorted locations for block " + b.toString() + "\n"); + int i = 0; + for (DatanodeInfo info : b.getLocations()) { + sortOutput.append(" #" + (i++) + " " + info + "\n"); + } } + + System.out.println(sortOutput); } CyclicIteration getDatanodeCyclicIteration( From 890e326b25c8423a918eba66c722915a78fa4466 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 10 Dec 2014 13:47:30 -0500 Subject: [PATCH 101/112] Revert "Add some logging for datanode selection" This reverts commit 220170e62c349f8bd4863ed3add03f35cdc3cb33. --- .../apache/hadoop/hdfs/DFSInputStream.java | 4 ---- .../blockmanagement/DatanodeManager.java | 22 ++----------------- 2 files changed, 2 insertions(+), 24 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index 3ea21ecb5095..c025ee8d46bd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -816,10 +816,6 @@ private DNAddrPair chooseDataNode(LocatedBlock block) throws IOException { while (true) { DatanodeInfo[] nodes = block.getLocations(); - String s = ""; - for (DatanodeInfo info : nodes) - s = s + " " + info.toString(); - System.out.printf("%d chooseDataNode: %s\n", Thread.currentThread().getId(), s); try { DatanodeInfo chosenNode = bestNode(nodes, deadNodes); final String dnAddr = diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index b2c03f973653..ff93d73d5385 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -329,25 +329,15 @@ public void sortLocatedBlocks(final String targethost, //sort the blocks // As it is possible for the separation of node manager and datanode, // here we should get node but not datanode only . - StringBuilder sortOutput = new StringBuilder(); - sortOutput.append("sortLocatedBlocks status:\n"); - Node client = getDatanodeByHost(targethost); if (client == null) { - sortOutput.append(" - client==null, targethost="+targethost+"\n"); List hosts = new ArrayList (1); hosts.add(targethost); String rName = dnsToSwitchMapping.resolve(hosts).get(0); - if (rName != null) { - sortOutput.append(" - rname = " + rName+"\n"); - sortOutput.append(" - create NodeBase: " + rName + NodeBase.PATH_SEPARATOR_STR + targethost + "\n"); + if (rName != null) client = new NodeBase(rName + NodeBase.PATH_SEPARATOR_STR + targethost); - } else { - sortOutput.append(" - rname == null " + rName+"\n"); - } } - - sortOutput.append(" - avoidStaleDataNodesForRead=="+avoidStaleDataNodesForRead+"\n"); + Comparator comparator = avoidStaleDataNodesForRead ? new DFSUtil.DecomStaleComparator(staleInterval) : DFSUtil.DECOM_COMPARATOR; @@ -356,15 +346,7 @@ public void sortLocatedBlocks(final String targethost, networktopology.pseudoSortByDistance(client, b.getLocations()); // Move decommissioned/stale datanodes to the bottom Arrays.sort(b.getLocations(), comparator); - - sortOutput.append(" - Sorted locations for block " + b.toString() + "\n"); - int i = 0; - for (DatanodeInfo info : b.getLocations()) { - sortOutput.append(" #" + (i++) + " " + info + "\n"); - } } - - System.out.println(sortOutput); } CyclicIteration getDatanodeCyclicIteration( From e7b60207fd8268b90a62350c3c53765dfea6c39c Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 23 Sep 2013 12:48:07 -0400 Subject: [PATCH 102/112] Applied XTrace patch Removed unnecessary import Instrumented the WritableRpcEngine. I'm not sure where this is used (as opposed to the Protobuf engine) but Todd's htrace instrumentation chose to instrument this class too, so I might as well include it. Add XTrace metadata to the data transfer header protos Start tracing commands when initiated from the filesystem. Changed the DataNode receiver to start traces or join traces. Added XTrace metadata to the block and packet transfer headers Instrumented the sender/receiver for downloading files from HDFS Add check for xtrace metadata Move start trace to the DataXceiver and add a little instrumentation for writeblock. Add XTrace instrumentation to pipeline write acks Instrument code that mirrors data when pipelining to in-place modify the xtrace metadata Removed unnecessary instrumentation Minor mistakes in the xtrace log statements Tiny bits of additional instrumentation Changed location of 'close' log messages, since close can be called multiple times, we only want it logged when the stream is first closed. Finished implementation of file writes from client side; mostly implementation in DFSOutputStream Instrumentation bugfix Instrumented a couple more protobuf messages, added a utility class to put/retrieve XTrace contexts from those protobuf messages, and modified a bunch of classes to call the utility methods rather than the protobuf newBuilder methods. Fixed a context joining problem in the block receiver Instrumented start and end of many DFSClient api function calls Added some resource tracing events Added an extra log statement to the DFSOutputStream Added log statement to indicate that we're forcing a namenode block update Added some instrumentation of some of the main locks used on the namenode. Removed some annoying unnecessary RPC log messages, and added a 'name' tag to name the spans in the rpc invoker. Changed the startTrace logpoints in DataXceiver to just logEvents. Starting traces here isn't such a good idea; background tasks end up accidentally starting traces when really we're not interestedin them. Use XTraceResourceTracing branching API to log the explicit computation boundaries when kicking off new threads. Also, add some names when setting XTraceContext to give spans names. Slightly modify server to extend the boundary of where the xtrace context is cleared. Removed all resource tracing stuff, which will be moved to a new branch. Added dependency to XResourceTracing Temporarily disabled code to propagate metadata between replicas, as it seems to be causing problems Fixed an XTrace logging bug Fix for the mysterious bug that was causing checksum inconsistencies. Root cause was a bug in HDFS source code unrelated to X-Trace instrumentation. Have fixed the bug. Some small instrumentation tweaks Added XResourceTracing as a bootclasspath option Added the necessary command line arguments to put xresourcetracing on the bootclasspath, but left them commented out for now We can now put xresourcetracing on the bootclasspath. It only takes effect if xresourcetracing was built to weave rt.jar Use a more accurate estimate for the PacketHeader size, now taking into account the fact that XTrace metadata can propagate options which might increase the PacketHeader size Removed some hard-coded xtrace context passing that is now handled generically by the AspectJ instrumentation Clear the thread context in lease renewer, which is a long-lived thread and shouldn't be attributed to the first task that kicks it off Fix to add causality when the sending thread has to wait to receive ACKs Fixed a bug - using less than instead of greater than Removed and modified some of the XTraceContext.startTrace events, because they were polluting trace tags and annoying the hell out of me Commented out the inclusion of resource tracing on the bootclasspath; it limits the classes that can be turned on and off in xtrace config. For now, we aren't using rt.jar instrumentation, and therefore no reason for these to be on the bootclasspath Commented out bootclasspath stuff, since for now we don't want or need it More network instrumentation for RPC calls Added a few entries to the default hdfs config; by default, there are a few things we want turned off Testing the write speed with a lower cache drop behind buffer lag Applied broken pom patch from HADOOP-10110; hasn't affected us thus far but sandbox build was failing Temporary instrumentation adding additional events to DataXceiver Undo previous commit Add some more temporary logging Removed some logging that was temporary Revert "Removed some logging that was temporary" This reverts commit d8d8727210e37d44bd13a3996b305ba4819b5e07. Migration from X-Trace 2.0 to X-Trace 3.0. Preliminary commit of untested instrumentation Small fix Peer cache is long lived, don't attribute to first task we see Whoops... didn't commit this properly... Add a few log messages to NativeIO, why not Instrument call queue in IPC Server Slight changes to the RPC server and client response sending/processing threads, to make sure the correct XTrace metadata is set at all times Add ability to have connection-per-client in a single process For now, comment out the random sleep if a complete call fails after writing a file, because it produces way too much arbitrary interference Add 5ms sleep instead of 400ms sleep Add instrumentation of DN heartbeats Bad import removed Instrument more background tasks Add a hacky addition to allow kinda throttling of background block replication Oh, and make methods static Fix divide by zero exception Try alternative approach to replication throttling using the balancer bandwidth Also default to large balancer bandwidth Use both approaches simultaneously! Removed balancer bandwidth - unnecessary Moved the AspectJ stuff to the root pom. I'm really not sure which one it's supposed to go in though. Indentation in pom Add showWeaveInfo=true to pom Add throttling points to HDFS Rename the throttling point on the RPC server Put servername in the call queue to allow multiple servers Fix up the propagation of X-Trace metadata from NodeManager to Containers (specifically, YarnChild and MRAppMaster containers Let the examples in the examples jar start X-Trace tasks, useful for debugging and verifying that everything is working Base64 encoding isn't quite compatible with HTTP headers because it uses a disallowed character '=' for padding. For now just switch to Base16, which is less efficient but only includes alphanumeric characters. Could tweak X-Trace base64 to not include padding but don't want to risk violating any existing instrumentation, no time to fix it Fixed up the inclusion of xtrace metadata in the shuffle header Put throttling queue only on NN server for now Only add call queue instrumentation for namenode for now Foolishly forgot to use base 16 decoding of XTrace header in shuffle handler Add throttling point to ShuffleHandler and to RawFileSystem Add some special handling for MR shuffle handler network output Remove previous commit instrumentation of shuffle handler, was too much Remove throttling point from local file system, put in spill thread Add some cpu tracking... test Put throttling points in new position remove spill throttler Share throttling point for ifile Do this manually... Manually disable datanode hostname check. Later versions of HDFS make this configurable. Add HDFS config option to specify whether to fadvise long files Add some logging for datanode selection Revert "Add some logging for datanode selection" This reverts commit 220170e62c349f8bd4863ed3add03f35cdc3cb33. --- hadoop-common-project/hadoop-auth/pom.xml | 15 +- .../src/main/bin/hadoop-config.cmd | 6 + .../src/main/bin/hadoop-config.sh | 4 + .../org/apache/hadoop/fs/FSOutputSummer.java | 1 + .../java/org/apache/hadoop/fs/FsShell.java | 11 + .../apache/hadoop/fs/RawLocalFileSystem.java | 3 +- .../apache/hadoop/io/nativeio/NativeIO.java | 8 +- .../java/org/apache/hadoop/ipc/Client.java | 56 +++- .../apache/hadoop/ipc/ProtobufRpcEngine.java | 45 ++- .../java/org/apache/hadoop/ipc/Server.java | 81 +++++- .../apache/hadoop/ipc/WritableRpcEngine.java | 58 +++- .../java/org/apache/hadoop/net/NetUtils.java | 30 +- .../org/apache/hadoop/util/ProgramDriver.java | 6 + .../org/apache/hadoop/util/ProtoUtil.java | 7 +- .../java/org/apache/hadoop/util/Shell.java | 16 ++ .../src/main/proto/RpcHeader.proto | 2 + hadoop-common-project/hadoop-nfs/pom.xml | 2 +- .../hadoop-hdfs/src/main/conf/hdfs-site.xml | 16 ++ .../hadoop/hdfs/BlockReaderFactory.java | 2 + .../org/apache/hadoop/hdfs/DFSClient.java | 16 +- .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 2 + .../apache/hadoop/hdfs/DFSOutputStream.java | 83 +++++- .../org/apache/hadoop/hdfs/LeaseRenewer.java | 7 + .../org/apache/hadoop/hdfs/PeerCache.java | 12 +- .../apache/hadoop/hdfs/RemoteBlockReader.java | 3 + .../hadoop/hdfs/RemoteBlockReader2.java | 18 +- .../hdfs/protocol/XTraceProtoUtils.java | 75 +++++ .../datatransfer/DataTransferProtoUtil.java | 13 +- .../protocol/datatransfer/PacketHeader.java | 39 ++- .../protocol/datatransfer/PacketReceiver.java | 49 +++- .../protocol/datatransfer/PipelineAck.java | 25 +- .../hdfs/protocol/datatransfer/Receiver.java | 20 +- .../hadoop/hdfs/server/balancer/Balancer.java | 2 + .../blockmanagement/DatanodeDescriptor.java | 43 ++- .../blockmanagement/DatanodeManager.java | 14 +- .../server/blockmanagement/Host2NodesMap.java | 37 ++- .../hdfs/server/datanode/BPServiceActor.java | 16 +- .../datanode/BlockPoolSliceStorage.java | 10 + .../hdfs/server/datanode/BlockReceiver.java | 50 +++- .../hdfs/server/datanode/BlockSender.java | 34 ++- .../hadoop/hdfs/server/datanode/DNConf.java | 4 + .../hadoop/hdfs/server/datanode/DataNode.java | 13 + .../hdfs/server/datanode/DataStorage.java | 9 + .../hdfs/server/datanode/DataXceiver.java | 53 +++- .../impl/FsDatasetAsyncDiskService.java | 11 +- .../fsdataset/impl/FsDatasetImpl.java | 8 +- .../hdfs/server/namenode/FSNamesystem.java | 89 +++++- .../src/main/proto/datatransfer.proto | 6 + .../hadoop/mapred/LocalContainerLauncher.java | 8 +- .../mapred/TaskAttemptListenerImpl.java | 10 +- .../org/apache/hadoop/mapred/YarnChild.java | 19 ++ .../jobhistory/JobHistoryEventHandler.java | 6 + .../hadoop/mapreduce/v2/app/MRAppMaster.java | 29 +- .../v2/app/TaskHeartbeatHandler.java | 7 +- .../hadoop/mapreduce/v2/app/job/Job.java | 2 + .../mapreduce/v2/app/job/impl/JobImpl.java | 99 ++++++- .../v2/app/job/impl/TaskAttemptImpl.java | 5 +- .../mapreduce/v2/app/job/impl/TaskImpl.java | 3 +- .../app/launcher/ContainerLauncherImpl.java | 13 +- .../v2/app/local/LocalContainerAllocator.java | 1 + .../mapreduce/v2/app/rm/RMCommunicator.java | 2 + .../v2/app/rm/RMContainerAllocator.java | 79 +++++- .../v2/app/rm/RMContainerRequestor.java | 62 ++++- .../v2/app/speculate/DefaultSpeculator.java | 2 +- .../apache/hadoop/mapreduce/v2/app/MRApp.java | 1 + .../hadoop/mapreduce/v2/app/MockJobs.java | 10 + .../v2/app/TestRuntimeEstimators.java | 10 + .../apache/hadoop/mapred/LocalJobRunner.java | 23 +- .../v2/api/records/TaskAttemptId.java | 14 + .../apache/hadoop/mapred/CleanupQueue.java | 1 - .../java/org/apache/hadoop/mapred/IFile.java | 14 +- .../org/apache/hadoop/mapred/IndexRecord.java | 53 +++- .../apache/hadoop/mapred/JobEndNotifier.java | 4 +- .../org/apache/hadoop/mapred/MapTask.java | 130 ++++++++- .../java/org/apache/hadoop/mapred/Merger.java | 6 +- .../org/apache/hadoop/mapred/ReduceTask.java | 10 +- .../org/apache/hadoop/mapred/SpillRecord.java | 4 +- .../java/org/apache/hadoop/mapred/Task.java | 72 ++++- .../mapred/lib/MultithreadedMapRunner.java | 19 +- .../org/apache/hadoop/mapreduce/Cluster.java | 8 +- .../java/org/apache/hadoop/mapreduce/Job.java | 41 ++- .../hadoop/mapreduce/JobSubmissionFiles.java | 10 +- .../apache/hadoop/mapreduce/JobSubmitter.java | 17 +- .../org/apache/hadoop/mapreduce/Mapper.java | 3 + .../org/apache/hadoop/mapreduce/Reducer.java | 3 + .../lib/map/MultithreadedMapper.java | 14 +- .../hadoop/mapreduce/task/reduce/Fetcher.java | 57 +++- .../task/reduce/InMemoryMapOutput.java | 16 +- .../mapreduce/task/reduce/MapOutput.java | 18 +- .../task/reduce/MergeManagerImpl.java | 86 +++++- .../mapreduce/task/reduce/MergeThread.java | 8 +- .../task/reduce/OnDiskMapOutput.java | 15 +- .../hadoop/mapreduce/task/reduce/Shuffle.java | 22 +- .../mapreduce/task/reduce/ShuffleHeader.java | 8 + .../task/reduce/ShuffleScheduler.java | 2 +- .../task/reduce/ShuffleSchedulerImpl.java | 14 +- .../hadoop/mapreduce/v2/hs/CompletedJob.java | 12 + .../hadoop/mapreduce/v2/hs/JobHistory.java | 2 + .../hadoop/mapreduce/v2/hs/PartialJob.java | 11 + .../hadoop/mapred/ResourceMgrDelegate.java | 9 +- .../org/apache/hadoop/mapred/YARNRunner.java | 1 + .../hadoop/mapred/FadvisedChunkedFile.java | 16 +- .../hadoop/mapred/FadvisedFileRegion.java | 16 +- .../apache/hadoop/mapred/ShuffleHandler.java | 25 +- .../org/apache/hadoop/examples/WordCount.java | 10 +- .../hadoop/yarn/api/records/ContainerId.java | 4 + .../yarn/api/records/LocalResource.java | 11 + .../yarn/api/records/ResourceRequest.java | 3 + .../src/main/proto/yarn_protos.proto | 3 + .../yarn/YarnUncaughtExceptionHandler.java | 10 + .../records/impl/pb/ContainerIdPBImpl.java | 21 ++ .../records/impl/pb/LocalResourcePBImpl.java | 17 ++ .../impl/pb/ResourceRequestPBImpl.java | 18 ++ .../hadoop/yarn/event/AbstractEvent.java | 19 ++ .../hadoop/yarn/event/AsyncDispatcher.java | 19 ++ .../org/apache/hadoop/yarn/event/Event.java | 2 + .../hadoop/yarn/state/StateMachine.java | 1 + .../yarn/state/StateMachineFactory.java | 256 ++++++++++++++++-- .../apache/hadoop/yarn/util/FSDownload.java | 18 +- .../yarn/server/utils/BuilderUtils.java | 3 + .../nodemanager/DefaultContainerExecutor.java | 11 +- .../server/nodemanager/DeletionService.java | 2 +- .../nodemanager/LinuxContainerExecutor.java | 13 + .../nodemanager/NodeStatusUpdaterImpl.java | 6 +- .../protocolrecords/LocalResourceStatus.java | 3 + .../impl/pb/LocalResourceStatusPBImpl.java | 18 ++ .../containermanager/AuxServices.java | 1 + .../ContainerManagerImpl.java | 13 +- .../application/ApplicationImpl.java | 13 +- .../container/ContainerImpl.java | 23 +- .../launcher/ContainerLaunch.java | 10 +- .../launcher/ContainersLauncher.java | 6 + .../localizer/ContainerLocalizer.java | 17 +- .../localizer/LocalResourceRequest.java | 8 + .../localizer/LocalResourcesTrackerImpl.java | 3 +- .../localizer/LocalizedResource.java | 5 +- .../ResourceLocalizationService.java | 32 ++- .../logaggregation/LogAggregationService.java | 1 + .../loghandler/NonAggregatingLogHandler.java | 1 + .../monitor/ContainersMonitorImpl.java | 12 +- ...rn_server_nodemanager_service_protos.proto | 1 + .../localizer/MockLocalResourceStatus.java | 5 + .../ApplicationMasterService.java | 4 + .../resourcemanager/ClientRMService.java | 8 +- .../resourcemanager/NodesListManager.java | 1 + .../server/resourcemanager/RMAppManager.java | 5 + .../resourcemanager/ResourceManager.java | 14 +- .../amlauncher/AMLauncher.java | 15 +- .../amlauncher/ApplicationMasterLauncher.java | 1 + .../resourcemanager/rmapp/RMAppEvent.java | 3 + .../resourcemanager/rmapp/RMAppImpl.java | 11 +- .../rmapp/attempt/RMAppAttemptImpl.java | 8 +- .../rmcontainer/RMContainerImpl.java | 4 +- .../resourcemanager/rmnode/RMNodeImpl.java | 17 +- .../scheduler/capacity/CapacityScheduler.java | 12 + .../scheduler/capacity/LeafQueue.java | 14 +- .../common/fica/FiCaSchedulerApp.java | 8 +- .../scheduler/fair/FairScheduler.java | 14 +- .../scheduler/fifo/FifoScheduler.java | 25 +- .../server/resourcemanager/NodeManager.java | 2 +- pom.xml | 56 ++++ 161 files changed, 2816 insertions(+), 307 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java diff --git a/hadoop-common-project/hadoop-auth/pom.xml b/hadoop-common-project/hadoop-auth/pom.xml index 5f474dd02125..75f0d64b55fb 100644 --- a/hadoop-common-project/hadoop-auth/pom.xml +++ b/hadoop-common-project/hadoop-auth/pom.xml @@ -52,12 +52,17 @@ org.mockito mockito-all test - + - org.mortbay.jetty - jetty - test - + org.mortbay.jetty + jetty-util + test + + + org.mortbay.jetty + jetty + test + javax.servlet servlet-api diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd index 3e6e4571a4eb..61689885df8c 100644 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd @@ -289,4 +289,10 @@ if not "%HADOOP_MAPRED_HOME%\%MAPRED_DIR%" == "%HADOOP_YARN_HOME%\%YARN_DIR%" ( set CLASSPATH=!CLASSPATH!;%HADOOP_MAPRED_HOME%\%MAPRED_DIR%\* ) +@rem XResourceTracing - add the XBootclasspath jars +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-resource-tracing-1.0.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\xtrace-2.1-20120824.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\log4j-1.2.17.jar +@rem set HADOOP_OPTS=%HADOOP_OPTS% -Xbootclasspath/p:%HADOOP_COMMON_HOME%\%HADOOP_COMMON_LIB_JARS_DIR%\aspectjrt-1.7.3.jar + :eof diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh index e5c40fcd6cf8..ce5768864a8c 100644 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh @@ -293,3 +293,7 @@ if [ "$HADOOP_CLASSPATH" != "" ]; then fi fi +#HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/xtrace-resource-tracing-1.0.jar" +#HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/xtrace-2.1-20120824.jar" +#HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/log4j-1.2.17.jar" +#HADOOP_OPTS="$HADOOP_OPTS -Xbootclasspath/p:${HADOOP_COMMON_LIB_JARS_DIR}/aspectjrt-1.7.3.jar" diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java index 691fbfbd16d3..4b7e545c047c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java @@ -103,6 +103,7 @@ public synchronized void write(byte b[], int off, int len) for (int n=0;n getProtocolMetaInfoProxy( private static class Invoker implements RpcInvocationHandler { private final Map returnTypes = new ConcurrentHashMap(); + private boolean isClientCached = true; private boolean isClosed = false; private final Client.ConnectionId remoteId; private final Client client; @@ -127,7 +133,11 @@ private Invoker(Class protocol, InetSocketAddress addr, private Invoker(Class protocol, Client.ConnectionId connId, Configuration conf, SocketFactory factory) { this.remoteId = connId; - this.client = CLIENTS.getClient(conf, factory, RpcResponseWrapper.class); + this.isClientCached = conf.getBoolean("xtrace.client.cached", true); + if (this.isClientCached) + this.client = CLIENTS.getClient(conf, factory, RpcResponseWrapper.class); + else + this.client = new Client(RpcResponseWrapper.class, conf, factory); this.protocolName = RPC.getProtocolName(protocol); this.clientProtocolVersion = RPC .getProtocolVersion(protocol); @@ -176,6 +186,11 @@ private RequestHeaderProto constructRpcRequestHeader(Method method) { @Override public Object invoke(Object proxy, Method method, Object[] args) throws ServiceException { + + xtrace.log("Invoking remote method "+method.getName(), "Protocol", this.protocolName, "ConnectionID", this.remoteId); + Context start_context = XTrace.get(); + try { // xtrace try + long startTime = 0; if (LOG.isDebugEnabled()) { startTime = Time.now(); @@ -241,14 +256,26 @@ public Object invoke(Object proxy, Method method, Object[] args) } catch (Throwable e) { throw new ServiceException(e); } + + XTrace.join(start_context); + xtrace.log("Client invocation of "+method.getName()+" successful"); + return returnMessage; + } catch (ServiceException e) {// xtrace catch + XTrace.join(start_context); + xtrace.log("Remote invocation of "+method.getName()+" failed due to exception: "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } } @Override public void close() throws IOException { if (!isClosed) { isClosed = true; - CLIENTS.stopClient(client); + if (isClientCached) + CLIENTS.stopClient(client); + else + client.stop(); } } @@ -563,6 +590,10 @@ public Writable call(RPC.Server server, String protocol, if (server.verbose) LOG.info("Call: protocol=" + protocol + ", method=" + methodName); + xtrace.log("Invoking method "+methodName, "Protocol", protocol, "Name", "RPC: "+methodName); + Context start_context = XTrace.get(); + try { // xtrace try + ProtoClassProtoImpl protocolImpl = getProtocolImpl(server, protoName, clientVersion); BlockingService service = (BlockingService) protocolImpl.protocolImpl; @@ -598,7 +629,15 @@ public Writable call(RPC.Server server, String protocol, } catch (Exception e) { throw e; } - return new RpcResponseWrapper(result); + + XTrace.join(start_context); + xtrace.log("Invocation of "+methodName+" completed, responding to client"); + return new RpcResponseWrapper(result); + } catch (Exception e) { // xtrace catch + XTrace.join(start_context); + xtrace.log("Failed to invoke method "+methodName+": "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index 5e32e7068bbe..2126f9bd589a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -18,6 +18,11 @@ package org.apache.hadoop.ipc; +import static org.apache.hadoop.ipc.RpcConstants.AUTHORIZATION_FAILED_CALL_ID; +import static org.apache.hadoop.ipc.RpcConstants.CONNECTION_CONTEXT_CALL_ID; +import static org.apache.hadoop.ipc.RpcConstants.CURRENT_VERSION; +import static org.apache.hadoop.ipc.RpcConstants.PING_CALL_ID; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -71,8 +76,6 @@ import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; -import static org.apache.hadoop.ipc.RpcConstants.*; - import org.apache.hadoop.ipc.ProtobufRpcEngine.RpcResponseMessageWrapper; import org.apache.hadoop.ipc.ProtobufRpcEngine.RpcResponseWrapper; import org.apache.hadoop.ipc.RPC.RpcInvoker; @@ -113,6 +116,13 @@ import com.google.protobuf.Message; import com.google.protobuf.Message.Builder; +import edu.brown.cs.systems.resourcethrottling.LocalThrottlingPoints; +import edu.brown.cs.systems.resourcetracing.resources.Network; +import edu.brown.cs.systems.resourcetracing.resources.QueueResource; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.Reporter.Utils; +import edu.brown.cs.systems.xtrace.XTrace; + /** An abstract IPC service. IPC calls take a single {@link Writable} as a * parameter, and return a {@link Writable} as their value. A service runs on * a port and is defined by a parameter class and a value class. @@ -235,7 +245,7 @@ public static RpcInvoker getRpcInvoker(RPC.RpcKind rpcKind) { return (val == null) ? null : val.rpcInvoker; } - + public static final XTrace.Logger xtrace = XTrace.getLogger(Server.class); public static final Log LOG = LogFactory.getLog(Server.class); public static final Log AUDITLOG = LogFactory.getLog("SecurityLogger."+Server.class.getName()); @@ -368,6 +378,8 @@ public static boolean isRpcInvocation() { private final boolean tcpNoDelay; // if T then disable Nagle's Algorithm volatile private boolean running = true; // true while server runs + + private QueueResource callQueueInstrumentation = null; // xresourcetracing instrumentation for the callqueue. could be better but will do for now private BlockingQueue callQueue; // queued calls private List connectionList = @@ -477,6 +489,10 @@ public static class Call { private ByteBuffer rpcResponse; // the response for this call private final RPC.RpcKind rpcKind; private final byte[] clientId; + + private Context start_context; // the X-Trace context this was received with + public long enqueue, dequeue, complete; // Timers for queue instrumentation; hacky but quick + private Context response_context; // the X-Trace context before sending the response public Call(int id, int retryCount, Writable param, Connection connection) { @@ -494,6 +510,8 @@ public Call(int id, int retryCount, Writable param, Connection connection, this.rpcResponse = null; this.rpcKind = kind; this.clientId = clientId; + Server.xtrace.log("Received RPC call", "CallID", id); + this.start_context = XTrace.get(); } @Override @@ -859,6 +877,7 @@ private class Responder extends Thread { @Override public void run() { LOG.info(getName() + ": starting"); + XTrace.stop(); SERVER.set(Server.this); try { doRunLoop(); @@ -987,6 +1006,10 @@ private boolean processResponse(LinkedList responseQueue, boolean done = false; // there is more data for this channel. int numElements = 0; Call call = null; + + if (!inHandler) + XTrace.stop(); + try { synchronized (responseQueue) { // @@ -1001,6 +1024,7 @@ private boolean processResponse(LinkedList responseQueue, // Extract the first call // call = responseQueue.removeFirst(); + XTrace.join(call.response_context); SocketChannel channel = call.connection.channel; if (LOG.isDebugEnabled()) { LOG.debug(getName() + ": responding to " + call); @@ -1013,6 +1037,7 @@ private boolean processResponse(LinkedList responseQueue, return true; } if (!call.rpcResponse.hasRemaining()) { + xtrace.log("Finished writing RPC response"); //Clear out the response buffer so it can be collected call.rpcResponse = null; call.connection.decRpcCount(); @@ -1026,6 +1051,7 @@ private boolean processResponse(LinkedList responseQueue, + " Wrote " + numBytes + " bytes."); } } else { + xtrace.log("Wrote partial RPC response, enqueueing for later finish"); // // If we were unable to write the entire response out, then // insert in Selector queue. @@ -1062,6 +1088,8 @@ private boolean processResponse(LinkedList responseQueue, done = true; // error. no more data for this channel. closeConnection(call.connection); } + if (!inHandler) + XTrace.stop(); } return done; } @@ -1782,6 +1810,10 @@ private void processOneRpc(byte[] buf) } checkRpcHeaders(header); + // XTrace: one of the few places in hdfs source code where we have to explicitly call resource instrumentation code + Network.Read.alreadyStarted(this); + Network.Read.alreadyFinished(this, buf.length); + if (callId < 0) { // callIds typically used during connection setup processRpcOutOfBandRequest(header, dis); } else if (!connectionContextRead) { @@ -1799,6 +1831,8 @@ private void processOneRpc(byte[] buf) ioe.getClass().getName(), ioe.getMessage()); responder.doRespond(call); throw wrse; + } finally { + XTrace.stop(); } } @@ -1809,6 +1843,9 @@ private void processOneRpc(byte[] buf) */ private void checkRpcHeaders(RpcRequestHeaderProto header) throws WrappedRpcServerException { + if (header.hasXtrace()) + XTrace.set(header.getXtrace().toByteArray()); + if (!header.hasRpcOp()) { String err = " IPC Server: No rpc op in rpcRequestHeader"; throw new WrappedRpcServerException( @@ -1870,6 +1907,9 @@ private void processRpcRequest(RpcRequestHeaderProto header, Call call = new Call(header.getCallId(), header.getRetryCount(), rpcRequest, this, ProtoUtil.convert(header.getRpcKind()), header .getClientId().toByteArray()); + if (callQueueInstrumentation!=null) + callQueueInstrumentation.enqueue(); + call.enqueue = System.nanoTime(); callQueue.put(call); // queue the call; maybe blocked here incRpcCount(); // Increment the rpc count } @@ -2012,8 +2052,16 @@ public void run() { ByteArrayOutputStream buf = new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE); while (running) { + XTrace.stop(); try { final Call call = callQueue.take(); // pop the queue; maybe blocked here + call.dequeue = System.nanoTime(); + XTrace.set(call.start_context); + if (callQueueInstrumentation!=null) + callQueueInstrumentation.starting(call.enqueue, call.dequeue); + + try { // xtrace try + if (LOG.isDebugEnabled()) { LOG.debug(getName() + ": " + call + " for RpcKind " + call.rpcKind); } @@ -2095,6 +2143,12 @@ public Writable run() throws Exception { } responder.doRespond(call); } + + } finally { // xtrace finally + call.complete = System.nanoTime(); + if (callQueueInstrumentation!=null) + callQueueInstrumentation.finished(call.enqueue, call.dequeue, call.complete); + } } catch (InterruptedException e) { if (running) { // unexpected -- log it LOG.info(getName() + " unexpectedly interrupted", e); @@ -2172,7 +2226,13 @@ protected Server(String bindAddress, int port, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT); } - this.callQueue = new LinkedBlockingQueue(maxQueueSize); + if ("NameNode".equals(Utils.getProcessName())) { + // Hack; put a throttling queue on NN only for now + this.callQueue = LocalThrottlingPoints.getThrottlingQueue(Utils.getProcessName()+"-"+serverName); + this.callQueueInstrumentation = new QueueResource("Server-"+System.identityHashCode(this)+"-callQueue", handlerCount); + } else { + this.callQueue = new LinkedBlockingQueue(maxQueueSize); + } this.maxIdleTime = 2 * conf.getInt( CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT); @@ -2291,6 +2351,16 @@ private void setupResponse(ByteArrayOutputStream responseBuf, headerBuilder.setRetryCount(call.retryCount); headerBuilder.setStatus(status); headerBuilder.setServerIpcVersionNum(CURRENT_VERSION); + if (XTrace.active()) + headerBuilder.setXtrace(ByteString.copyFrom(XTrace.bytes())); + + /* X-Trace: we have to send in the response the last event in the server + * before the data is sent, and this is not it, there can be more events + * later, related to enqueuing and sending this call. To log them correctly + * here, we'd have to write the metadata after all these events, maybe + * having the X-Trace metadata as a writable after the response. Alternatively, + * we could use the clock within a span to log these. + */ if (status == RpcStatusProto.SUCCESS) { RpcResponseHeaderProto header = headerBuilder.build(); @@ -2340,6 +2410,7 @@ private void setupResponse(ByteArrayOutputStream responseBuf, wrapWithSasl(responseBuf, call); } call.setResponse(ByteBuffer.wrap(responseBuf.toByteArray())); + call.response_context = XTrace.get(); } /** @@ -2579,7 +2650,7 @@ private int channelWrite(WritableByteChannel channel, * This is a wrapper around {@link ReadableByteChannel#read(ByteBuffer)}. * If the amount of data is large, it writes to channel in smaller chunks. * This is to avoid jdk from creating many direct buffers as the size of - * ByteBuffer increases. There should not be any performance degredation. + * ByteBuffer increases. There should not be any performance degradation. * * @see ReadableByteChannel#read(ByteBuffer) */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java index 817b743b5689..bc1a566f8b1f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java @@ -18,33 +18,40 @@ package org.apache.hadoop.ipc; -import java.lang.reflect.Proxy; -import java.lang.reflect.Method; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.lang.reflect.InvocationTargetException; - +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; import java.net.InetSocketAddress; -import java.io.*; import javax.net.SocketFactory; -import org.apache.commons.logging.*; - -import org.apache.hadoop.io.*; +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.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.ObjectWritable; +import org.apache.hadoop.io.UTF8; +import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.ipc.Client.ConnectionId; import org.apache.hadoop.ipc.RPC.RpcInvoker; -import org.apache.hadoop.ipc.VersionedProtocol; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.Time; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.*; + +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; /** An RpcEngine implementation for Writable data. */ @InterfaceStability.Evolving public class WritableRpcEngine implements RpcEngine { + private static final XTrace.Logger xtrace = XTrace.getLogger(RPC.class); private static final Log LOG = LogFactory.getLog(RPC.class); //writableRpcVersion should be updated if there is a change @@ -228,13 +235,27 @@ public Object invoke(Object proxy, Method method, Object[] args) startTime = Time.now(); } + xtrace.log("RPC Client invoking remote method "+method.getName(), "ConnectionID", this.remoteId); + Context start_context = XTrace.get(); + try { // xtrace try + ObjectWritable value = (ObjectWritable) client.call(RPC.RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId); if (LOG.isDebugEnabled()) { long callTime = Time.now() - startTime; LOG.debug("Call: " + method.getName() + " " + callTime); } + + XTrace.join(start_context); + xtrace.log("Client invocation of "+method.getName()+" successful"); + return value.get(); + + } catch (Exception e) {// xtrace catch + XTrace.join(start_context); + xtrace.log("Remote invocation of "+method.getName()+" failed due to exception: "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } } /* close the IPC client that's responsible for this invoker's RPCs */ @@ -420,6 +441,10 @@ public Writable call(org.apache.hadoop.ipc.RPC.Server server, Invocation call = (Invocation)rpcRequest; if (server.verbose) log("Call: " + call); + + xtrace.log("Invoking method", "Method", call.getMethodName()); + Context start_context = XTrace.get(); + try { // xtrace try // Verify writable rpc version if (call.getRpcVersion() != writableRpcVersion) { @@ -493,6 +518,9 @@ public Writable call(org.apache.hadoop.ipc.RPC.Server server, processingTime); if (server.verbose) log("Return: "+value); + XTrace.join(start_context); + xtrace.log("Invocation of method completed, responding to client", "Method", call.getMethodName()); + return new ObjectWritable(method.getReturnType(), value); } catch (InvocationTargetException e) { @@ -512,7 +540,13 @@ public Writable call(org.apache.hadoop.ipc.RPC.Server server, ioe.setStackTrace(e.getStackTrace()); throw ioe; } - } + + } catch (IOException e) { // xtrace catch + XTrace.join(start_context); + xtrace.log("Failed to invoke method "+call.getMethodName()+": "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } + } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java index 8595f87b9e8c..dc9738bc9afe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java @@ -22,6 +22,7 @@ import java.io.OutputStream; import java.lang.reflect.Constructor; import java.net.BindException; +import java.net.ConnectException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.NetworkInterface; @@ -34,12 +35,17 @@ import java.net.URI; import java.net.URISyntaxException; import java.net.UnknownHostException; -import java.net.ConnectException; import java.nio.channels.SocketChannel; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Map.Entry; -import java.util.regex.Pattern; -import java.util.*; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.regex.Pattern; import javax.net.SocketFactory; @@ -58,9 +64,13 @@ import com.google.common.base.Preconditions; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; + @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @InterfaceStability.Unstable public class NetUtils { + private static final XTrace.Logger xtrace = XTrace.getLogger(NetUtils.class); private static final Log LOG = LogFactory.getLog(NetUtils.class); private static Map hostToResolved = @@ -509,6 +519,10 @@ public static void connect(Socket socket, if (socket == null || endpoint == null || timeout < 0) { throw new IllegalArgumentException("Illegal argument for connect()"); } + + xtrace.log("Connecting to remote", "Socket", socket.toString(), "Timeout", timeout); + Context start_context = XTrace.get(); + try { // xtrace try SocketChannel ch = socket.getChannel(); @@ -546,6 +560,14 @@ public static void connect(Socket socket, "Localhost targeted connection resulted in a loopback. " + "No daemon is listening on the target port."); } + + XTrace.join(start_context); + xtrace.log("Connected to remote host"); + } catch (IOException e) { // xtrace catch + XTrace.join(start_context); + xtrace.log("Failed to connect to remote host: "+e.getClass().getName(), "Message", e.getMessage()); + throw e; + } } /** @@ -904,4 +926,4 @@ public static int getFreeSocketPort() { } return port; } -} +} \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java index e52dacccbb4f..956fb58b673e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java @@ -26,6 +26,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import edu.brown.cs.systems.xtrace.XTrace; + /** A driver that is used to run programs added to it */ @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @@ -135,6 +137,10 @@ public int run(String[] args) printUsage(programs); return -1; } + + XTrace.startTask(true); + XTrace.setTenantClass(55); + XTrace.getLogger("ProgramDriver").log("Executing example program", "ProgramName", args[0]); // Remove the leading argument and call main String[] new_args = new String[args.length - 1]; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java index 79f8692842dc..c062459c9d66 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java @@ -24,12 +24,15 @@ import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto; import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.UserInformationProto; -import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*; +import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcKindProto; +import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto; import org.apache.hadoop.security.SaslRpcServer.AuthMethod; import org.apache.hadoop.security.UserGroupInformation; import com.google.protobuf.ByteString; +import edu.brown.cs.systems.xtrace.XTrace; + public abstract class ProtoUtil { /** @@ -165,6 +168,8 @@ public static RpcRequestHeaderProto makeRpcRequestHeader(RPC.RpcKind rpcKind, RpcRequestHeaderProto.Builder result = RpcRequestHeaderProto.newBuilder(); result.setRpcKind(convert(rpcKind)).setRpcOp(operation).setCallId(callId) .setRetryCount(retryCount).setClientId(ByteString.copyFrom(uuid)); + if (XTrace.active()) + result.setXtrace(ByteString.copyFrom(XTrace.bytes())); return result.build(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java index 49edea0b142d..3cd4b10cd021 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java @@ -32,6 +32,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import edu.brown.cs.systems.xtrace.XTrace; + /** * A base class for running a Unix command. * @@ -384,6 +386,17 @@ private void runCommand() throws IOException { if (environment != null) { builder.environment().putAll(this.environment); } + builder.environment().putAll(XTrace.environment()); + +// // TODO: implement this in new xtrace +// // put xtrace context if there is one, merging as appropriate +// XTraceMetadata child_process_xtrace = null; +// if (XTraceContext.isValid()) { +// child_process_xtrace = XTraceContext.startChildProcess(); +// builder.environment().put(XTraceContext.XTRACE_CONTEXT_ENV_VARIABLE, XTraceContext.logMerge().toString()); +// builder.environment().put(XTraceContext.XTRACE_SUBPROCESS_ENV_VARIABLE, child_process_xtrace.toString()); +// } + if (dir != null) { builder.directory(this.dir); } @@ -445,6 +458,9 @@ public void run() { } // wait for the process to finish and check the exit code exitCode = process.waitFor(); + +// // TODO: join the xtrace process +// XTraceContext.joinChildProcess(child_process_xtrace); try { // make sure that the error thread exits errThread.join(); diff --git a/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto b/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto index 673883b23a59..44759d15cdd1 100644 --- a/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto +++ b/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto @@ -67,6 +67,7 @@ message RpcRequestHeaderProto { // the header for the RpcRequest // clientId + callId uniquely identifies a request // retry count, 1 means this is the first retry optional sint32 retryCount = 5 [default = -1]; + optional bytes xtrace = 6; // X-Trace context } @@ -132,6 +133,7 @@ message RpcResponseHeaderProto { optional RpcErrorCodeProto errorDetail = 6; // in case of error optional bytes clientId = 7; // Globally unique client ID optional sint32 retryCount = 8 [default = -1]; + optional bytes xtrace = 9; // X-Trace context } message RpcSaslProto { diff --git a/hadoop-common-project/hadoop-nfs/pom.xml b/hadoop-common-project/hadoop-nfs/pom.xml index 324c9830890d..c01962818b2b 100644 --- a/hadoop-common-project/hadoop-nfs/pom.xml +++ b/hadoop-common-project/hadoop-nfs/pom.xml @@ -92,7 +92,7 @@ com.google.guava guava - 11.0.2 + 17.0 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml index 50ec1460bd60..776c540e7cde 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml @@ -17,5 +17,21 @@ + + dfs.namenode.fs-limits.min-block-size + 65536 + + + dfs.datanode.readahead.bytes + 0 + + + dfs.datanode.drop.cache.behind.writes + true + + + dfs.datanode.sync.behind.writes + true + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java index 197ab23b29fa..09655ec63a11 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; @@ -177,6 +178,7 @@ private static BlockReaderLocal newShortCircuitBlockReader( new DataInputStream(peer.getInputStream()); BlockOpResponseProto resp = BlockOpResponseProto.parseFrom( PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(resp); DomainSocket sock = peer.getDomainSocket(); switch (resp.getStatus()) { case SUCCESS: diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index c6aef2c929d0..6e9af4390867 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -61,9 +61,7 @@ import java.io.OutputStream; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.NetworkInterface; import java.net.Socket; -import java.net.SocketException; import java.net.SocketAddress; import java.net.URI; import java.net.UnknownHostException; @@ -99,6 +97,7 @@ import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Options.ChecksumOpt; import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.VolumeId; import org.apache.hadoop.fs.permission.FsPermission; @@ -112,16 +111,17 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; +import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.Op; @@ -130,8 +130,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; -import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.protocolPB.PBHelper; +import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; @@ -162,6 +162,7 @@ import com.google.common.base.Preconditions; import com.google.common.net.InetAddresses; + /******************************************************** * DFSClient can connect to a Hadoop Filesystem and * perform basic file tasks. It uses the ClientProtocol @@ -1165,6 +1166,7 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum, public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) throws IOException, UnresolvedLinkException { checkOpen(); + //XTraceContext.startTrace("DFSClient", "Open", "open", src); // Get block info from namenode return new DFSInputStream(this, src, buffersize, verifyChecksum); } @@ -1336,6 +1338,7 @@ public DFSOutputStream create(String src, + favoredNodes[i].getPort(); } } + //XTraceContext.startTrace("DFSClient", "Create", "create", src); final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this, src, masked, flag, createParent, replication, blockSize, progress, buffersize, dfsClientConf.createChecksum(checksumOpt), favoredNodeStrs); @@ -1358,6 +1361,7 @@ private DFSOutputStream primitiveAppend(String src, EnumSet flag, } return null; } + //XTraceContext.startTrace("DFSClient", "Append", "append", src); return callAppend(stat, src, buffersize, progress); } return null; @@ -1382,6 +1386,7 @@ public DFSOutputStream primitiveCreate(String src, CreateFlag.validate(flag); DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress); if (result == null) { + //XTraceContext.startTrace("DFSClient", "Create", "create", src); DataChecksum checksum = dfsClientConf.createChecksum(checksumOpt); result = DFSOutputStream.newStreamForCreate(this, src, absPermission, flag, createParent, replication, blockSize, progress, buffersize, @@ -1462,6 +1467,7 @@ private DFSOutputStream callAppend(HdfsFileStatus stat, String src, public HdfsDataOutputStream append(final String src, final int buffersize, final Progressable progress, final FileSystem.Statistics statistics ) throws IOException { + //XTraceContext.startTrace("DFSClient", "Append", "append", src); final DFSOutputStream out = append(src, buffersize, progress); return new HdfsDataOutputStream(out, statistics, out.getInitialLen()); } @@ -1789,6 +1795,7 @@ private static MD5MD5CRC32FileChecksum getFileChecksum(String src, final BlockOpResponseProto reply = BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(reply); if (reply.getStatus() != Status.SUCCESS) { if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) { @@ -1971,6 +1978,7 @@ private static Type inferChecksumTypeByReading( new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName, 0, 1, true); final BlockOpResponseProto reply = BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(reply); if (reply.getStatus() != Status.SUCCESS) { if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 0bc1c25ee8ed..5cae58c8e7db 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -93,6 +93,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final boolean DFS_DATANODE_SYNC_BEHIND_WRITES_DEFAULT = false; public static final String DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY = "dfs.datanode.drop.cache.behind.reads"; public static final boolean DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT = false; + public static final String DFS_DATANODE_FADVISE_SEQUENTIAL = "dfs.datanode.fadvise.sequential.reads"; + public static final boolean DFS_DATANODE_FADVISE_SEQUENTIAL_DEFAULT = true; public static final String DFS_DATANODE_USE_DN_HOSTNAME = "dfs.datanode.use.datanode.hostname"; public static final boolean DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT = false; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 0d6747f03a9a..b9a7d753acfd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -58,9 +58,10 @@ import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; -import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor; +import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; @@ -91,6 +92,10 @@ import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; + + /**************************************************************** * DFSOutputStream creates files from a stream of bytes. @@ -116,6 +121,7 @@ ****************************************************************/ @InterfaceAudience.Private public class DFSOutputStream extends FSOutputSummer implements Syncable { + private final XTrace.Logger xtrace = XTrace.getLogger(DFSOutputStream.class); private final DFSClient dfsClient; private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB private Socket s; @@ -148,6 +154,8 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable { private final short blockReplication; // replication factor of file private boolean shouldSyncBlock = false; // force blocks to disk upon close + private Context lastAckedXTraceContext = null; + private class Packet { long seqno; // sequencenumber of buffer in block long offsetInBlock; // offset in block @@ -178,6 +186,8 @@ private class Packet { int dataPos; private static final long HEART_BEAT_SEQNO = -1L; + + private Context xtrace_context = null; /** * Create a heartbeat packet. @@ -215,6 +225,15 @@ private class Packet { dataStart = checksumStart + (chunksPerPkt * checksum.getChecksumSize()); dataPos = dataStart; maxChunks = chunksPerPkt; + rememberXTraceContext(); + } + + void joinXTraceContext() { + XTrace.join(xtrace_context); + } + + void rememberXTraceContext() { + xtrace_context = XTrace.get(); } void writeData(byte[] inarray, int off, int len) { @@ -350,7 +369,7 @@ public DatanodeInfo load(DatanodeInfo key) throws Exception { private boolean isHflushed = false; /** Append on an existing block? */ private final boolean isAppend; - + /** * Default construction for file create */ @@ -426,6 +445,7 @@ private void setFavoredNodes(String[] favoredNodes) { private void initDataStreaming() { this.setName("DataStreamer for file " + src + " block " + block); + xtrace.log("Kicking off response processor"); response = new ResponseProcessor(nodes); response.start(); stage = BlockConstructionStage.DATA_STREAMING; @@ -435,6 +455,7 @@ private void endBlock() { if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Closing old block " + block); } + xtrace.log("Closing block", "BlockName", block.getBlockName()); this.setName("DataStreamer for file " + src); closeResponder(); closeStream(); @@ -494,10 +515,13 @@ public void run() { } // get packet to be sent. if (dataQueue.isEmpty()) { + xtrace.log("Sending heartbeat packet"); one = new Packet(); // heartbeat packet } else { one = dataQueue.getFirst(); // regular data packet } + one.joinXTraceContext(); + } assert one != null; @@ -506,12 +530,14 @@ public void run() { if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Allocating new block"); } + xtrace.log("Allocating new block"); nodes = nextBlockOutputStream(src); initDataStreaming(); } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) { if(DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Append to block " + block); } + xtrace.log("Appending to block", "BlockName", block.getBlockName()); setupPipelineForAppendOrRecovery(); initDataStreaming(); } @@ -543,11 +569,14 @@ public void run() { stage = BlockConstructionStage.PIPELINE_CLOSE; } + xtrace.log("Sending packet"); + // send the packet synchronized (dataQueue) { // move packet from dataQueue to ackQueue if (!one.isHeartbeatPacket()) { dataQueue.removeFirst(); + one.rememberXTraceContext(); ackQueue.addLast(one); dataQueue.notifyAll(); } @@ -570,6 +599,9 @@ public void run() { } lastPacket = Time.now(); + xtrace.log("Packet sent"); + one.rememberXTraceContext(); + if (one.isHeartbeatPacket()) { //heartbeat packet } @@ -606,6 +638,7 @@ public void run() { } } catch (Throwable e) { DFSClient.LOG.warn("DataStreamer Exception", e); + xtrace.log("DataStreamerException", "Message", e.getMessage()); if (e instanceof IOException) { setLastException((IOException)e); } @@ -627,7 +660,7 @@ private void closeInternal() { dataQueue.notifyAll(); } } - + /* * close both streamer and DFSOutputStream, should be called only * by an external thread and only after all data to be sent has @@ -705,7 +738,6 @@ private class ResponseProcessor extends Daemon { @Override public void run() { - setName("ResponseProcessor for block " + block); PipelineAck ack = new PipelineAck(); @@ -719,6 +751,8 @@ public void run() { } long seqno = ack.getSeqno(); + ack.joinXtraceContext(); + xtrace.log("Processing ACK", "seqno", seqno); // processes response status from datanodes. for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) { final Status reply = ack.getReply(i); @@ -742,6 +776,7 @@ public void run() { synchronized (dataQueue) { one = ackQueue.getFirst(); } + one.joinXTraceContext(); if (one.seqno != seqno) { throw new IOException("Responseprocessor: Expecting seqno " + " for block " + block + @@ -751,8 +786,10 @@ public void run() { // update bytesAcked block.setNumBytes(one.getLastByteOffsetBlock()); + xtrace.log("Packet acknowledged"); synchronized (dataQueue) { lastAckedSeqno = seqno; + lastAckedXTraceContext = XTrace.get(); ackQueue.removeFirst(); dataQueue.notifyAll(); } @@ -768,6 +805,7 @@ public void run() { } DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception " + " for block " + block, e); + xtrace.log("Exception processing responses", "Message", e.getMessage()); responderClosed = true; } } @@ -778,6 +816,7 @@ void close() { responderClosed = true; this.interrupt(); } + } // If this stream has encountered any errors so far, shutdown @@ -899,6 +938,7 @@ private void addDatanode2ExistingPipeline() throws IOException { * - Append/Create: * + no transfer, let NameNode replicates the block. */ + xtrace.log("Adding DataNode to Existing Pipeline"); if (!isAppend && lastAckedSeqno < 0 && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) { //no data have been written @@ -956,6 +996,7 @@ private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets, //ack BlockOpResponseProto response = BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(response); if (SUCCESS != response.getStatus()) { throw new IOException("Failed to add a datanode"); } @@ -1102,6 +1143,7 @@ private DatanodeInfo[] nextBlockOutputStream(String client) throws IOException { // private boolean createBlockOutputStream(DatanodeInfo[] nodes, long newGS, boolean recoveryFlag) { + Status pipelineStatus = SUCCESS; String firstBadLink = ""; if (DFSClient.LOG.isDebugEnabled()) { @@ -1150,6 +1192,7 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, long newGS, PBHelper.vintPrefixed(blockReplyStream)); pipelineStatus = resp.getStatus(); firstBadLink = resp.getFirstBadLink(); + XTraceProtoUtils.join(resp); if (pipelineStatus != SUCCESS) { if (pipelineStatus == Status.ERROR_ACCESS_TOKEN) { @@ -1454,6 +1497,8 @@ private void queueCurrentPacket() { if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Queued packet " + currentPacket.seqno); } + xtrace.log("Packet added to output queue"); + currentPacket.rememberXTraceContext(); currentPacket = null; dataQueue.notifyAll(); } @@ -1462,7 +1507,12 @@ private void queueCurrentPacket() { private void waitAndQueueCurrentPacket() throws IOException { synchronized (dataQueue) { // If queue is full, then wait till we have enough space + boolean first = true; while (!closed && dataQueue.size() + ackQueue.size() > MAX_PACKETS) { + if (first) { + xtrace.log("Waiting for packets to be acked", "lastAckedSeqno", lastAckedSeqno); + first = false; + } try { dataQueue.wait(); } catch (InterruptedException e) { @@ -1476,6 +1526,8 @@ private void waitAndQueueCurrentPacket() throws IOException { Thread.currentThread().interrupt(); break; } + if (dataQueue.size() + ackQueue.size() <= MAX_PACKETS) + XTrace.join(lastAckedXTraceContext); } checkClosed(); queueCurrentPacket(); @@ -1513,6 +1565,8 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] che ", chunksPerPacket=" + chunksPerPacket + ", bytesCurBlock=" + bytesCurBlock); } + xtrace.log("Allocating new packet", "seqno", currentPacket.seqno); + currentPacket.rememberXTraceContext(); } currentPacket.writeChecksum(checksum, 0, cklen); @@ -1520,6 +1574,10 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] che currentPacket.numChunks++; bytesCurBlock += len; + // Add in the current xtrace context + currentPacket.joinXTraceContext(); + currentPacket.rememberXTraceContext(); + // If packet is full, enqueue it for transmission // if (currentPacket.numChunks == currentPacket.maxChunks || @@ -1551,6 +1609,7 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] che // indicate the end of block and reset bytesCurBlock. // if (bytesCurBlock == blockSize) { + xtrace.log("Sending empty packet to indicate end of block"); currentPacket = new Packet(0, 0, bytesCurBlock); currentPacket.lastPacketInBlock = true; currentPacket.syncBlock = shouldSyncBlock; @@ -1618,6 +1677,12 @@ public void hsync(EnumSet syncFlags) throws IOException { */ private void flushOrSync(boolean isSync, EnumSet syncFlags) throws IOException { + if (isSync) + xtrace.log("hysnc - Syncing all data to datanode disks"); + else + xtrace.log("hflush - Flushing all data to datanode buffers"); + + dfsClient.checkOpen(); checkClosed(); try { @@ -1725,6 +1790,7 @@ private void flushOrSync(boolean isSync, EnumSet syncFlags) throw interrupt; } catch (IOException e) { DFSClient.LOG.warn("Error while syncing", e); + xtrace.log("Error while syncing", "Message", e.getMessage()); synchronized (this) { if (!closed) { lastException = new IOException("IOException flush:" + e); @@ -1785,10 +1851,13 @@ private void waitForAckedSeqno(long seqno) throws IOException { if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("Waiting for ack for: " + seqno); } + xtrace.log("Waiting for ack", "seqno", seqno); synchronized (dataQueue) { while (!closed) { checkClosed(); if (lastAckedSeqno >= seqno) { + XTrace.join(lastAckedXTraceContext); + xtrace.log("Ack received, continuing", "lastAckedSeqno", lastAckedSeqno); break; } try { @@ -1853,6 +1922,8 @@ public synchronized void close() throws IOException { } try { + xtrace.log("Closing stream"); + flushBuffer(); // flush from all upper layers if (currentPacket != null) { @@ -1874,6 +1945,7 @@ public synchronized void close() throws IOException { dfsClient.endFileLease(src); } finally { closed = true; + xtrace.log("Stream closed"); } } @@ -1896,8 +1968,9 @@ private void completeFile(ExtendedBlock last) throws IOException { DFSClient.LOG.info(msg); throw new IOException(msg); } + // Jon: reduce this from 400 to 5 just so as not to taint our exps try { - Thread.sleep(400); + Thread.sleep(5); if (Time.now() - localstart > 5000) { DFSClient.LOG.info("Could not complete " + src + " retrying..."); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java index 8f3b8fb0e325..aaf8ac2b50de 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java @@ -35,8 +35,11 @@ import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; + import com.google.common.annotations.VisibleForTesting; +import edu.brown.cs.systems.xtrace.XTrace; + /** *

    * Used by {@link DFSClient} for renewing file-being-written leases @@ -69,6 +72,7 @@ *

    */ class LeaseRenewer { + static final XTrace.Logger xtrace = XTrace.getLogger(LeaseRenewer.class); static final Log LOG = LogFactory.getLog(LeaseRenewer.class); static final long LEASE_RENEWER_GRACE_DEFAULT = 60*1000L; @@ -290,6 +294,9 @@ synchronized void put(final String src, final DFSOutputStream out, daemon = new Daemon(new Runnable() { @Override public void run() { + xtrace.log("Lease renewer daemon started"); + XTrace.stop(); // don't let the task id leak to lease renewer. + // if lease renewer tracing desired, should start a new task here and maybe log an event. try { if (LOG.isDebugEnabled()) { LOG.debug("Lease renewer daemon for " + clientsString() diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java index 424b641c8c39..9cd9cebd6c07 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/PeerCache.java @@ -22,17 +22,20 @@ import java.util.List; import java.util.Map.Entry; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.LinkedListMultimap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Time; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.LinkedListMultimap; + +import edu.brown.cs.systems.xtrace.XTrace; + /** * A cache of input stream sockets to Data Node. */ @@ -131,6 +134,7 @@ private synchronized void startExpiryDaemon() { daemon = new Daemon(new Runnable() { @Override public void run() { + XTrace.stop(); // Long-lived, don't attribute to a task try { PeerCache.this.run(); } catch(InterruptedException e) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index 61ce3eab157a..a9e393fd0671 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; @@ -383,6 +384,7 @@ public static RemoteBlockReader newBlockReader(String file, DatanodeID datanodeID, PeerCache peerCache) throws IOException { + // in and out will be closed when sock is closed (by the caller) final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(peer.getOutputStream())); @@ -398,6 +400,7 @@ public static RemoteBlockReader newBlockReader(String file, BlockOpResponseProto status = BlockOpResponseProto.parseFrom( PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(status); RemoteBlockReader2.checkSuccess(status, peer, block, file); ReadOpChecksumInfoProto checksumInfo = status.getReadOpChecksumInfo(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java index 1ba6b55da8f3..736e7d4e65d6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java @@ -32,15 +32,14 @@ import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; -import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; @@ -50,6 +49,8 @@ import com.google.common.annotations.VisibleForTesting; +import edu.brown.cs.systems.xtrace.XTrace; + /** * This is a wrapper around connection to datanode * and understands checksum, offset etc. @@ -79,6 +80,7 @@ @InterfaceAudience.Private public class RemoteBlockReader2 implements BlockReader { + static final XTrace.Logger xtrace = XTrace.getLogger(RemoteBlockReader2.class); static final Log LOG = LogFactory.getLog(RemoteBlockReader2.class); final private Peer peer; @@ -213,6 +215,7 @@ private void readNextPacket() throws IOException { // If we've now satisfied the whole client read, read one last packet // header, which should be empty if (bytesNeededToFinish <= 0) { + xtrace.log("Block finished, reading trailing empty packet"); readTrailingEmptyPacket(); if (verifyChecksum) { sendReadResult(Status.CHECKSUM_OK); @@ -321,7 +324,7 @@ void sendReadResult(Status statusCode) { static void writeReadResult(OutputStream out, Status statusCode) throws IOException { - ClientReadStatusProto.newBuilder() + XTraceProtoUtils.newClientReadStatusProtoBuilder() .setStatus(statusCode) .build() .writeDelimitedTo(out); @@ -376,6 +379,9 @@ public static BlockReader newBlockReader(String file, String clientName, Peer peer, DatanodeID datanodeID, PeerCache peerCache) throws IOException { + xtrace.log("Reading remote block", "file", file, "BlockName", block.getBlockName()); + try { // xtrace try + // in and out will be closed when sock is closed (by the caller) final DataOutputStream out = new DataOutputStream(new BufferedOutputStream( peer.getOutputStream())); @@ -389,6 +395,7 @@ public static BlockReader newBlockReader(String file, BlockOpResponseProto status = BlockOpResponseProto.parseFrom( PBHelper.vintPrefixed(in)); + XTraceProtoUtils.join(status); checkSuccess(status, peer, block, file); ReadOpChecksumInfoProto checksumInfo = status.getReadOpChecksumInfo(); @@ -409,6 +416,11 @@ public static BlockReader newBlockReader(String file, return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(), checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID, peerCache); + + } catch (IOException e) { + xtrace.log("IOException reading remote block", "Message", e.getMessage()); + throw e; + } } static void checkSuccess( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java new file mode 100644 index 000000000000..2275f9ba707d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/XTraceProtoUtils.java @@ -0,0 +1,75 @@ +package org.apache.hadoop.hdfs.protocol; + +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto; + +import com.google.protobuf.ByteString; + +import edu.brown.cs.systems.xtrace.XTrace; + +/** + * Contains some utility functions for XTrace instrumentation. Saves having to repeat + * instrumentation in loads of places in the code. + * @author jon + * + */ +public class XTraceProtoUtils { + + /** + * Shortcut method to create a new builder, then insert the current XTraceContext into it + * @return + */ + public static BlockOpResponseProto.Builder newBlockOpResponseProtoBuilder() { + BlockOpResponseProto.Builder b = BlockOpResponseProto.newBuilder(); + setXtrace(b); + return b; + } + + /** + * If the current XTraceContext is valid, sets it in the provided builder + * @param builder + */ + public static void setXtrace(BlockOpResponseProto.Builder builder) { + if (XTrace.active()) + builder.setXtrace(ByteString.copyFrom(XTrace.bytes())); + } + + /** + * Joins an XTrace context if this message contains one + * @param p + */ + public static void join(BlockOpResponseProto p) { + if (p.hasXtrace()) + XTrace.join(p.getXtrace().toByteArray()); + } + + /** + * Shortcut method to create a new builder, then insert the current XTraceContext into it + * @return + */ + public static ClientReadStatusProto.Builder newClientReadStatusProtoBuilder() { + ClientReadStatusProto.Builder b = ClientReadStatusProto.newBuilder(); + setXtrace(b); + return b; + } + + /** + * If the current XTraceContext is valid, sets it in the provided builder + * @param builder + */ + public static void setXtrace(ClientReadStatusProto.Builder builder) { + if (XTrace.active()) + builder.setXtrace(ByteString.copyFrom(XTrace.bytes())); + } + + /** + * Joins an XTrace context if this message contains one + * @param p + */ + public static void join(ClientReadStatusProto p) { + if (p.hasXtrace()) + XTrace.join(p.getXtrace().toByteArray()); + } + + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java index 6be3810c9187..d08d2158957f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java @@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; @@ -31,6 +32,10 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; +import com.google.protobuf.ByteString; + +import edu.brown.cs.systems.xtrace.XTrace; + /** * Static utilities for dealing with the protocol buffers used by the @@ -78,9 +83,11 @@ static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk, static BaseHeaderProto buildBaseHeader(ExtendedBlock blk, Token blockToken) { - return BaseHeaderProto.newBuilder() + Builder header = BaseHeaderProto.newBuilder() .setBlock(PBHelper.convert(blk)) - .setToken(PBHelper.convert(blockToken)) - .build(); + .setToken(PBHelper.convert(blockToken)); + if (XTrace.active()) + header.setXtrace(ByteString.copyFrom(XTrace.bytes())); + return header.build(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java index c9966a71a50d..e7aadeb8fbc2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java @@ -28,10 +28,13 @@ import org.apache.hadoop.hdfs.util.ByteBufferOutputStream; import com.google.common.base.Preconditions; -import com.google.common.primitives.Shorts; import com.google.common.primitives.Ints; +import com.google.common.primitives.Shorts; +import com.google.protobuf.ByteString; import com.google.protobuf.InvalidProtocolBufferException; +import edu.brown.cs.systems.xtrace.XTrace; + /** * Header data for each packet that goes through the read/write pipelines. * Includes all of the information about the packet, excluding checksums and @@ -51,14 +54,18 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class PacketHeader { - private static final int MAX_PROTO_SIZE = - PacketHeaderProto.newBuilder() - .setOffsetInBlock(0) - .setSeqno(0) - .setLastPacketInBlock(false) - .setDataLen(0) - .setSyncBlock(false) - .build().getSerializedSize(); + private static final XTrace.Logger xtrace = XTrace.getLogger(PacketHeader.class); + private static final int MAX_PROTO_SIZE; + static { + MAX_PROTO_SIZE = PacketHeaderProto.newBuilder() + .setOffsetInBlock(0) + .setSeqno(0) + .setLastPacketInBlock(false) + .setDataLen(0) + .setSyncBlock(false) + .setXtrace(ByteString.copyFrom(XTrace.XTRACE_BYTES_EXAMPLE)) + .build().getSerializedSize(); + } public static final int PKT_LENGTHS_LEN = Ints.BYTES + Shorts.BYTES; public static final int PKT_MAX_HEADER_LEN = @@ -82,6 +89,11 @@ public PacketHeader(int packetLen, long offsetInBlock, long seqno, .setSeqno(seqno) .setLastPacketInBlock(lastPacketInBlock) .setDataLen(dataLen); + + if (XTrace.active()) { + xtrace.log("Constructing packet header"); + builder.setXtrace(ByteString.copyFrom(XTrace.bytesBounded())); + } if (syncBlock) { // Only set syncBlock if it is specified. @@ -117,6 +129,15 @@ public int getPacketLen() { public boolean getSyncBlock() { return proto.getSyncBlock(); } + + public boolean hasXTraceContext() { + return proto.hasXtrace(); + } + + public void joinXTraceContext() { + if (proto.hasXtrace()) + XTrace.join(proto.getXtrace().toByteArray()); + } @Override public String toString() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java index cc2d17974a4e..05fd1e0b5efd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java @@ -33,6 +33,9 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; +import edu.brown.cs.systems.xtrace.XTrace; + + /** * Class to handle reading packets one-at-a-time from the wire. * These packets are used both for reading and writing data to/from @@ -47,11 +50,12 @@ public class PacketReceiver implements Closeable { */ private static final int MAX_PACKET_SIZE = 16 * 1024 * 1024; + static XTrace.Logger xtrace = XTrace.getLogger(PacketReceiver.class); static Log LOG = LogFactory.getLog(PacketReceiver.class); private static final DirectBufferPool bufferPool = new DirectBufferPool(); private final boolean useDirectBuffers; - + /** * The entirety of the most recently read packet. * The first PKT_LENGTHS_LEN bytes of this buffer are the @@ -127,6 +131,10 @@ private void doRead(ReadableByteChannel ch, InputStream in) // CHECKSUMS: the crcs for the data chunk. May be missing if // checksums were not requested // DATA the actual block data + + xtrace.log("Reading packet"); + try { // xtrace try + Preconditions.checkState(curHeader == null || !curHeader.isLastPacketInBlock()); curPacketBuf.clear(); @@ -178,7 +186,9 @@ private void doRead(ReadableByteChannel ch, InputStream in) if (curHeader == null) { curHeader = new PacketHeader(); } - curHeader.setFieldsFromData(dataPlusChecksumLen, headerBuf); + curHeader.setFieldsFromData(payloadLen, headerBuf); + curHeader.joinXTraceContext(); + xtrace.log("Finished reading packet"); // Compute the sub-slices of the packet int checksumLen = dataPlusChecksumLen - curHeader.getDataLen(); @@ -189,6 +199,11 @@ private void doRead(ReadableByteChannel ch, InputStream in) } reslicePacket(headerLen, checksumLen, curHeader.getDataLen()); + + } catch (IOException e) { // xtrace catch + xtrace.log("Exception reading packet", "Message", e.getMessage()); + throw e; + } } /** @@ -197,9 +212,39 @@ private void doRead(ReadableByteChannel ch, InputStream in) public void mirrorPacketTo(DataOutputStream mirrorOut) throws IOException { Preconditions.checkState(!useDirectBuffers, "Currently only supported for non-direct buffers"); + + xtrace.log("Mirroring packet"); + try { // xtrace try + + updateHeaderXTrace(); mirrorOut.write(curPacketBuf.array(), curPacketBuf.arrayOffset(), curPacketBuf.remaining()); + + xtrace.log("Packet mirrored successfully"); + } catch (IOException e) { // xtrace catch + xtrace.log("Exception writing block to mirror", "Message", e.getMessage()); + } + } + + /** + * This updates the XTrace metadata in the packet header to the current context + */ + private void updateHeaderXTrace() { + // Only update context if there was a previous one, and we assume they have the exact + // same length, so we can just drop in a new packet header. + if (XTrace.active() && curHeader.hasXTraceContext()) { + PacketHeader newHeader = new PacketHeader(curHeader.getPacketLen(), curHeader.getOffsetInBlock(), + curHeader.getSeqno(), curHeader.isLastPacketInBlock(), curHeader.getDataLen(), + curHeader.getSyncBlock()); + int priorPosition = curPacketBuf.position(); + int priorLimit = curPacketBuf.limit(); + curPacketBuf.position(0); + curPacketBuf.limit(newHeader.getSerializedSize()); + newHeader.putInBuffer(curPacketBuf); + curPacketBuf.position(priorPosition); + curPacketBuf.limit(priorLimit); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java index b743e29f217b..34a9097e7501 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java @@ -28,14 +28,20 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.Builder; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; +import com.google.protobuf.ByteString; import com.google.protobuf.TextFormat; +import edu.brown.cs.systems.xtrace.XTrace; + /** Pipeline Acknowledgment **/ @InterfaceAudience.Private @InterfaceStability.Evolving public class PipelineAck { + + private static final XTrace.Logger xtrace = XTrace.getLogger(PipelineAck.class); PipelineAckProto proto; public final static long UNKOWN_SEQNO = -2; @@ -59,11 +65,15 @@ public PipelineAck(long seqno, Status[] replies) { * @param downstreamAckTimeNanos ack RTT in nanoseconds, 0 if no next DN in pipeline */ public PipelineAck(long seqno, Status[] replies, long downstreamAckTimeNanos) { - proto = PipelineAckProto.newBuilder() + Builder builder = PipelineAckProto.newBuilder() .setSeqno(seqno) .addAllStatus(Arrays.asList(replies)) - .setDownstreamAckTimeNanos(downstreamAckTimeNanos) - .build(); + .setDownstreamAckTimeNanos(downstreamAckTimeNanos); + if (XTrace.active()) { + xtrace.log("creating pipelined ack"); + builder.setXtrace(ByteString.copyFrom(XTrace.bytes())); + } + proto = builder.build(); } /** @@ -97,6 +107,15 @@ public Status getReply(int i) { public long getDownstreamAckTimeNanos() { return proto.getDownstreamAckTimeNanos(); } + + public boolean hasXtrace() { + return proto.hasXtrace(); + } + + public void joinXtraceContext() { + if (proto.hasXtrace()) + XTrace.join(proto.getXtrace().toByteArray()); + } /** * Check if this ack contains error status diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index 910938ab44e0..e131588a2b13 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -17,23 +17,26 @@ */ package org.apache.hadoop.hdfs.protocol.datatransfer; -import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto; +import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; import java.io.DataInputStream; import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; +import edu.brown.cs.systems.xtrace.XTrace; + /** Receiver */ @InterfaceAudience.Private @InterfaceStability.Evolving @@ -84,10 +87,16 @@ protected final void processOp(Op op) throws IOException { throw new IOException("Unknown op " + op + " in data stream"); } } + + private static void XTraceJoinStart(String type, BaseHeaderProto header) { + if (header!=null && header.hasXtrace()) + XTrace.join(header.getXtrace().toByteArray()); + } /** Receive OP_READ_BLOCK */ private void opReadBlock() throws IOException { OpReadBlockProto proto = OpReadBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoinStart("ReadBlock", proto.getHeader().getBaseHeader()); readBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -99,6 +108,7 @@ private void opReadBlock() throws IOException { /** Receive OP_WRITE_BLOCK */ private void opWriteBlock(DataInputStream in) throws IOException { final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoinStart("WriteBlock", proto.getHeader().getBaseHeader()); writeBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -115,6 +125,7 @@ private void opWriteBlock(DataInputStream in) throws IOException { private void opTransferBlock(DataInputStream in) throws IOException { final OpTransferBlockProto proto = OpTransferBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoinStart("TransferBlock", proto.getHeader().getBaseHeader()); transferBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), @@ -125,6 +136,7 @@ private void opTransferBlock(DataInputStream in) throws IOException { private void opRequestShortCircuitFds(DataInputStream in) throws IOException { final OpRequestShortCircuitAccessProto proto = OpRequestShortCircuitAccessProto.parseFrom(vintPrefixed(in)); + XTraceJoinStart("RequestShortCircuitFds", proto.getHeader()); requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken()), proto.getMaxVersion()); @@ -133,6 +145,7 @@ private void opRequestShortCircuitFds(DataInputStream in) throws IOException { /** Receive OP_REPLACE_BLOCK */ private void opReplaceBlock(DataInputStream in) throws IOException { OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoinStart("ReplaceBlock", proto.getHeader()); replaceBlock(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken()), proto.getDelHint(), @@ -142,6 +155,7 @@ private void opReplaceBlock(DataInputStream in) throws IOException { /** Receive OP_COPY_BLOCK */ private void opCopyBlock(DataInputStream in) throws IOException { OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in)); + XTraceJoinStart("CopyBlock", proto.getHeader()); copyBlock(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken())); } @@ -149,7 +163,7 @@ private void opCopyBlock(DataInputStream in) throws IOException { /** Receive OP_BLOCK_CHECKSUM */ private void opBlockChecksum(DataInputStream in) throws IOException { OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in)); - + XTraceJoinStart("BlockChecksum", proto.getHeader()); blockChecksum(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken())); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java index cac8bf31bc78..d0b18665c645 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java @@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; +import org.apache.hadoop.hdfs.protocol.XTraceProtoUtils; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; @@ -392,6 +393,7 @@ private void sendRequest(DataOutputStream out) throws IOException { private void receiveResponse(DataInputStream in) throws IOException { BlockOpResponseProto response = BlockOpResponseProto.parseFrom( vintPrefixed(in)); + XTraceProtoUtils.join(response); if (response.getStatus() != Status.SUCCESS) { if (response.getStatus() == Status.ERROR_ACCESS_TOKEN) throw new IOException("block move failed due to access token error"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java index 56ac12f5ffe3..1f375fc8cd84 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java @@ -23,6 +23,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Queue; +import java.util.Random; import java.util.Set; import java.util.TreeSet; @@ -71,7 +72,7 @@ private static class BlockQueue { synchronized boolean offer(E e) { return blockq.offer(e); } - + /** Dequeue */ synchronized List poll(int numBlocks) { if (numBlocks <= 0 || blockq.isEmpty()) { @@ -404,6 +405,46 @@ int getNumberOfBlocksToBeInvalidated() { public List getReplicationCommand(int maxTransfers) { return replicateBlocks.poll(maxTransfers); } + + private static final Random r = new Random(); + + public List getReplicationCommand(int maxTransfers, long maxBytes) { + long blocksRemainingforReplication = 0; + long totalBytes = 0; + long totalBlocks = 0; + List results = null; + synchronized(replicateBlocks) { // Jon: terrible but quick + blocksRemainingforReplication = replicateBlocks.blockq.size(); + while (totalBytes < maxBytes && totalBlocks < maxTransfers) { + // Decide whether we're allowed the next block. Shouldn't be probabilitic, but easiest for now + BlockTargetPair next = replicateBlocks.blockq.peek(); + if (next==null) + break; + long nextsize = next.block.getNumBytes(); + if (totalBytes + nextsize > maxBytes) { + double probability = (maxBytes - totalBytes) / (double) nextsize; + if (r.nextDouble() > probability) + break; + } + + // Get the next block + next = replicateBlocks.blockq.poll(); + if (next==null) // shouldn't happen, but just in case + break; + + // Add the block + if (results==null) + results = new ArrayList(); + results.add(next); + + // Increment counters + totalBytes+=next.block.getNumBytes(); + totalBlocks++; + } + } + System.out.println("Replication: toReplicate="+blocksRemainingforReplication + " maxBlocks="+maxTransfers + " maxBytes="+maxBytes + " blocks="+totalBlocks+" bytes="+totalBytes); + return results; + } public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) { List blocks = recoverBlocks.poll(maxTransfers); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index 90541351dc50..ff93d73d5385 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -1138,20 +1138,22 @@ public List getDatanodeListForReport( * @return boolean true if name resolution successful or address is loopback */ private static boolean isNameResolved(InetAddress address) { - String hostname = address.getHostName(); - String ip = address.getHostAddress(); - return !hostname.equals(ip) || address.isLoopbackAddress(); +// String hostname = address.getHostName(); +// String ip = address.getHostAddress(); +// return !hostname.equals(ip) || address.isLoopbackAddress(); + return true; } private void setDatanodeDead(DatanodeDescriptor node) { node.setLastUpdate(0); } - /** Handle heartbeat from datanodes. */ + /** Handle heartbeat from datanodes. + * @param maxBytes */ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, final String blockPoolId, long capacity, long dfsUsed, long remaining, long blockPoolUsed, - int xceiverCount, int maxTransfers, int failedVolumes + int xceiverCount, int maxTransfers, long maxBytes, int failedVolumes ) throws IOException { synchronized (heartbeatManager) { synchronized (datanodeMap) { @@ -1223,7 +1225,7 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, final List cmds = new ArrayList(); //check pending replication List pendingList = nodeinfo.getReplicationCommand( - maxTransfers); + maxTransfers, maxBytes); if (pendingList != null) { cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId, pendingList)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java index 6f9049a960c1..1170de0f562f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/Host2NodesMap.java @@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.DFSUtil; + /** A map from host names to datanode descriptors. */ @InterfaceAudience.Private @InterfaceStability.Evolving @@ -34,6 +35,22 @@ class Host2NodesMap { private HashMap map = new HashMap(); private ReadWriteLock hostmapLock = new ReentrantReadWriteLock(); + + private void lockReadLock() { + hostmapLock.readLock().lock(); + } + + private void unlockReadLock() { + hostmapLock.readLock().unlock(); + } + + private void lockWriteLock() { + hostmapLock.writeLock().lock(); + } + + private void unlockWriteLock() { + hostmapLock.writeLock().unlock(); + } /** Check if node is already in the map. */ boolean contains(DatanodeDescriptor node) { @@ -42,7 +59,7 @@ boolean contains(DatanodeDescriptor node) { } String ipAddr = node.getIpAddr(); - hostmapLock.readLock().lock(); + lockReadLock(); try { DatanodeDescriptor[] nodes = map.get(ipAddr); if (nodes != null) { @@ -53,7 +70,7 @@ boolean contains(DatanodeDescriptor node) { } } } finally { - hostmapLock.readLock().unlock(); + unlockReadLock(); } return false; } @@ -62,7 +79,7 @@ boolean contains(DatanodeDescriptor node) { * return true if the node is added; false otherwise. */ boolean add(DatanodeDescriptor node) { - hostmapLock.writeLock().lock(); + lockWriteLock(); try { if (node==null || contains(node)) { return false; @@ -82,7 +99,7 @@ boolean add(DatanodeDescriptor node) { map.put(ipAddr, newNodes); return true; } finally { - hostmapLock.writeLock().unlock(); + unlockWriteLock(); } } @@ -95,7 +112,7 @@ boolean remove(DatanodeDescriptor node) { } String ipAddr = node.getIpAddr(); - hostmapLock.writeLock().lock(); + lockWriteLock(); try { DatanodeDescriptor[] nodes = map.get(ipAddr); @@ -128,7 +145,7 @@ boolean remove(DatanodeDescriptor node) { return true; } } finally { - hostmapLock.writeLock().unlock(); + unlockWriteLock(); } } @@ -141,7 +158,7 @@ DatanodeDescriptor getDatanodeByHost(String ipAddr) { return null; } - hostmapLock.readLock().lock(); + lockReadLock(); try { DatanodeDescriptor[] nodes = map.get(ipAddr); // no entry @@ -155,7 +172,7 @@ DatanodeDescriptor getDatanodeByHost(String ipAddr) { // more than one node return nodes[DFSUtil.getRandom().nextInt(nodes.length)]; } finally { - hostmapLock.readLock().unlock(); + unlockReadLock(); } } @@ -170,7 +187,7 @@ public DatanodeDescriptor getDatanodeByXferAddr(String ipAddr, return null; } - hostmapLock.readLock().lock(); + lockReadLock(); try { DatanodeDescriptor[] nodes = map.get(ipAddr); // no entry @@ -184,7 +201,7 @@ public DatanodeDescriptor getDatanodeByXferAddr(String ipAddr, } return null; } finally { - hostmapLock.readLock().unlock(); + unlockReadLock(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 75f42f959d76..ee5e398a44e5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -58,6 +58,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Maps; +import edu.brown.cs.systems.resourcetracing.backgroundtasks.HDFSBackgroundTask; +import edu.brown.cs.systems.xtrace.XTrace; + /** * A thread per active or standby namenode to perform: *
      @@ -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..11d49cae73ee 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,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.BlockingService; +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 +222,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 +1548,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 +1642,8 @@ public void run() { IOUtils.closeStream(out); IOUtils.closeStream(in); IOUtils.closeSocket(sock); + + HDFSBackgroundTask.REPLICATION.end(System.nanoTime() - begin); } } } @@ -1945,11 +1954,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/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(); private final List appRsrcs = new ArrayList(); + + private Collection xtrace_localizedresources = new HashSet(); public ContainerImpl(Configuration conf, Dispatcher dispatcher, ContainerLaunchContext launchContext, Credentials creds, @@ -131,7 +136,7 @@ public ContainerImpl(Configuration conf, Dispatcher dispatcher, private static StateMachineFactory stateMachineFactory = - new StateMachineFactory(ContainerState.NEW) + new StateMachineFactory(ContainerState.NEW, StateMachineFactory.Trace.KEEPALIVE) // From NEW State .addTransition(ContainerState.NEW, EnumSet.of(ContainerState.LOCALIZING, ContainerState.LOCALIZED, @@ -197,19 +202,19 @@ ContainerEventType.KILL_CONTAINER, new KillTransition()) .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_SUCCESS, ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS, - new ExitedWithSuccessTransition(true)) + new ExitedWithSuccessTransition(true), StateMachineFactory.Trace.ALWAYS) .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE, ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, - new ExitedWithFailureTransition(true)) + new ExitedWithFailureTransition(true), StateMachineFactory.Trace.ALWAYS) .addTransition(ContainerState.RUNNING, ContainerState.RUNNING, ContainerEventType.UPDATE_DIAGNOSTICS_MSG, - UPDATE_DIAGNOSTICS_TRANSITION) + UPDATE_DIAGNOSTICS_TRANSITION, StateMachineFactory.Trace.ALWAYS) .addTransition(ContainerState.RUNNING, ContainerState.KILLING, - ContainerEventType.KILL_CONTAINER, new KillTransition()) + ContainerEventType.KILL_CONTAINER, new KillTransition(), StateMachineFactory.Trace.ALWAYS) .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE, ContainerEventType.CONTAINER_KILLED_ON_REQUEST, - new KilledExternallyTransition()) + new KilledExternallyTransition(), StateMachineFactory.Trace.ALWAYS) // From CONTAINER_EXITED_WITH_SUCCESS State .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE, @@ -599,9 +604,12 @@ public ContainerState transition(ContainerImpl container, return ContainerState.LOCALIZING; } container.localizedResources.put(rsrcEvent.getLocation(), syms); + container.xtrace_localizedresources.add(XTrace.get()); if (!container.pendingResources.isEmpty()) { return ContainerState.LOCALIZING; } + for (Context ctx : container.xtrace_localizedresources) + XTrace.join(ctx); container.dispatcher.getEventHandler().handle( new ContainersLauncherEvent(container, ContainersLauncherEventType.LAUNCH_CONTAINER)); @@ -838,6 +846,7 @@ public void transition(ContainerImpl container, ContainerEvent event) { @Override public void handle(ContainerEvent event) { + event.joinContext(); try { 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/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java index 3f54ea4ba43a..597b92102f45 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java @@ -71,6 +71,8 @@ import org.apache.hadoop.yarn.util.Apps; import org.apache.hadoop.yarn.util.ConverterUtils; +import edu.brown.cs.systems.xtrace.XTrace; + public class ContainerLaunch implements Callable { private static final Log LOG = LogFactory.getLog(ContainerLaunch.class); @@ -98,6 +100,8 @@ public class ContainerLaunch implements Callable { private final LocalDirsHandlerService dirsHandler; + private edu.brown.cs.systems.xtrace.Context xtrace_context; + public ContainerLaunch(Context context, Configuration configuration, Dispatcher dispatcher, ContainerExecutor exec, Application app, Container container, LocalDirsHandlerService dirsHandler) { @@ -114,11 +118,15 @@ public ContainerLaunch(Context context, Configuration configuration, this.maxKillWaitTime = conf.getLong(YarnConfiguration.NM_PROCESS_KILL_WAIT_MS, YarnConfiguration.DEFAULT_NM_PROCESS_KILL_WAIT_MS); + xtrace_context = XTrace.get(); } - + @Override @SuppressWarnings("unchecked") // dispatcher not typed public Integer call() { + + XTrace.set(xtrace_context); + final ContainerLaunchContext launchContext = container.getLaunchContext(); final Map> localResources = container.getLocalizedResources(); 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/launcher/ContainersLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java index 4e374fd5b54d..815772aded09 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java @@ -45,6 +45,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.brown.cs.systems.xtrace.XTrace; + /** * The launcher for the containers. This service should be started only after * the {@link ResourceLocalizationService} is started as it depends on creation @@ -54,6 +56,7 @@ public class ContainersLauncher extends AbstractService implements EventHandler { + private static final XTrace.Logger xtrace = XTrace.getLogger(ContainersLauncher.class); private static final Log LOG = LogFactory.getLog(ContainersLauncher.class); private final Context context; @@ -109,11 +112,13 @@ protected void serviceStop() throws Exception { @Override public void handle(ContainersLauncherEvent event) { + event.joinContext(); // TODO: ContainersLauncher launches containers one by one!! Container container = event.getContainer(); ContainerId containerId = container.getContainerId(); switch (event.getType()) { case LAUNCH_CONTAINER: + xtrace.log("Launching container"); Application app = context.getApplications().get( containerId.getApplicationAttemptId().getApplicationId()); @@ -126,6 +131,7 @@ public void handle(ContainersLauncherEvent event) { launch)); break; case CLEANUP_CONTAINER: + xtrace.log("Cleanup container"); RunningContainer rContainerDatum = running.remove(containerId); if (rContainerDatum == null) { // Container not launched. So nothing needs to be done. 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/localizer/ContainerLocalizer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java index 66f21f6a5714..33f9769c249c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java @@ -65,14 +65,16 @@ import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus; import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenIdentifier; -import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.FSDownload; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.brown.cs.systems.xtrace.XTrace; + public class ContainerLocalizer { + static final XTrace.Logger xtrace = XTrace.getLogger(ContainerLocalizer.class); static final Log LOG = LogFactory.getLog(ContainerLocalizer.class); public static final String FILECACHE = "filecache"; @@ -225,6 +227,7 @@ protected void localizeFiles(LocalizationProtocol nodemanager, CompletionService cs, UserGroupInformation ugi) throws IOException { while (true) { + XTrace.stop(); try { LocalizerStatus status = createStatus(); LocalizerHeartbeatResponse response = nodemanager.heartbeat(status); @@ -232,11 +235,13 @@ protected void localizeFiles(LocalizationProtocol nodemanager, case LIVE: List newRsrcs = response.getResourceSpecs(); for (ResourceLocalizationSpec newRsrc : newRsrcs) { + newRsrc.getResource().joinContext(); if (!pendingResources.containsKey(newRsrc.getResource())) { pendingResources.put(newRsrc.getResource(), cs.submit(download( new Path(newRsrc.getDestinationDirectory().getFile()), newRsrc.getResource(), ugi))); } + XTrace.stop(); } break; case DIE: @@ -282,6 +287,9 @@ private LocalizerStatus createStatus() throws InterruptedException { if (fPath.isDone()) { try { Path localPath = fPath.get(); +// // TODO: deal with this properly +// XTraceContext.joinObject(localPath); + stat.rememberContext(); stat.setLocalPath( ConverterUtils.getYarnUrlFromPath(localPath)); stat.setLocalSize( @@ -300,6 +308,7 @@ private LocalizerStatus createStatus() throws InterruptedException { stat.setStatus(ResourceStatusType.FETCH_PENDING); } currentResources.add(stat); + XTrace.stop(); } LocalizerStatus status = recordFactory.newRecordInstance(LocalizerStatus.class); @@ -317,6 +326,7 @@ public static void main(String[] argv) throws Throwable { // MKDIR $x/$user/appcache/$appid/filecache // LOAD $x/$user/appcache/$appid/appTokens try { + xtrace.log("ContainerLocalizer launched..."); String user = argv[0]; String appId = argv[1]; String locId = argv[2]; @@ -339,7 +349,10 @@ public static void main(String[] argv) throws Throwable { new ContainerLocalizer(FileContext.getLocalFSFileContext(), user, appId, locId, localDirs, RecordFactoryProvider.getRecordFactory(null)); - System.exit(localizer.runLocalization(nmAddr)); + int retCode = localizer.runLocalization(nmAddr); +// // TODO: Xtrace deal with this +// XTraceContext.joinParentProcess(); + System.exit(retCode); } catch (Throwable e) { // Print error to stdout so that LCE can use it. e.printStackTrace(System.out); 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/localizer/LocalResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java index 70bead7320a4..72140f9bed6e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java @@ -191,4 +191,12 @@ public String toString() { sb.append(getPattern()).append(" }"); return sb.toString(); } + + @Override + public void rememberContext() { + } + + @Override + 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/containermanager/localizer/LocalResourcesTrackerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java index dfbeb449349f..1eb1406d929b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java @@ -107,6 +107,7 @@ public LocalResourcesTrackerImpl(String user, Dispatcher dispatcher, */ @Override public synchronized void handle(ResourceEvent event) { + event.joinContext(); LocalResourceRequest req = event.getLocalResourceRequest(); LocalizedResource rsrc = localrsrc.get(req); switch (event.getType()) { @@ -317,4 +318,4 @@ public long nextUniqueNumber() { public LocalizedResource getLocalizedResource(LocalResourceRequest request) { return localrsrc.get(request); } -} \ No newline at end of file +} 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/localizer/LocalizedResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java index f49e942ffd21..67f5d2114cfc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java @@ -71,7 +71,7 @@ public class LocalizedResource implements EventHandler { private static final StateMachineFactory stateMachineFactory = new StateMachineFactory(ResourceState.INIT) + ResourceEventType,ResourceEvent>(ResourceState.INIT, StateMachineFactory.Trace.KEEPALIVE) // From INIT (ref == 0, awaiting req) .addTransition(ResourceState.INIT, ResourceState.DOWNLOADING, @@ -81,7 +81,7 @@ ResourceEventType.REQUEST, new FetchResourceTransition()) .addTransition(ResourceState.DOWNLOADING, ResourceState.DOWNLOADING, ResourceEventType.REQUEST, new FetchResourceTransition()) // TODO: Duplicate addition!! .addTransition(ResourceState.DOWNLOADING, ResourceState.LOCALIZED, - ResourceEventType.LOCALIZED, new FetchSuccessTransition()) + ResourceEventType.LOCALIZED, new FetchSuccessTransition(), StateMachineFactory.Trace.ALWAYS) .addTransition(ResourceState.DOWNLOADING,ResourceState.DOWNLOADING, ResourceEventType.RELEASE, new ReleaseTransition()) .addTransition(ResourceState.DOWNLOADING, ResourceState.FAILED, @@ -180,6 +180,7 @@ public void unlock() { @Override public void handle(ResourceEvent event) { try { + event.joinContext(); this.writeLock.lock(); Path resourcePath = event.getLocalResourceRequest().getPath(); 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/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java index 1fdb0821708b..a89f0aeae42f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java @@ -31,6 +31,7 @@ import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -81,8 +82,8 @@ import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; -import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService; import org.apache.hadoop.yarn.server.nodemanager.DeletionService.FileDeletionTask; +import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService; import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol; import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec; import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus; @@ -119,9 +120,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; + public class ResourceLocalizationService extends CompositeService implements EventHandler, LocalizationProtocol { + private static final XTrace.Logger xtrace = XTrace.getLogger(ResourceLocalizationService.class); private static final Log LOG = LogFactory.getLog(ResourceLocalizationService.class); public static final String NM_PRIVATE_DIR = "nmPrivate"; public static final FsPermission NM_PRIVATE_PERM = new FsPermission((short) 0700); @@ -300,6 +305,7 @@ public void serviceStop() throws Exception { @Override public void handle(LocalizationEvent event) { + event.joinContext(); // TODO: create log dir as $logdir/$user/$appId switch (event.getType()) { case INIT_APPLICATION_RESOURCES: @@ -364,6 +370,8 @@ private void handleInitContainerResources( c.getUser(), c.getContainerId(), c.getCredentials()); Map> rsrcs = rsrcReqs.getRequestedResources(); + Context startCtx = XTrace.get(); + Collection endCtxs = new HashSet(); for (Map.Entry> e : rsrcs.entrySet()) { LocalResourcesTracker tracker = @@ -371,9 +379,13 @@ private void handleInitContainerResources( c.getContainerId().getApplicationAttemptId() .getApplicationId()); for (LocalResourceRequest req : e.getValue()) { + XTrace.set(startCtx); tracker.handle(new ResourceRequestEvent(req, e.getKey(), ctxt)); + endCtxs.add(XTrace.get()); } } + for (Context ctx : endCtxs) + XTrace.join(ctx); } private void handleCacheCleanup(LocalizationEvent event) { @@ -392,19 +404,26 @@ private void handleCacheCleanup(LocalizationEvent event) { @SuppressWarnings("unchecked") private void handleCleanupContainerResources( ContainerLocalizationCleanupEvent rsrcCleanup) { + rsrcCleanup.joinContext(); Container c = rsrcCleanup.getContainer(); Map> rsrcs = rsrcCleanup.getResources(); + Context start_context = XTrace.get(); + Collection end_context = new HashSet(); for (Map.Entry> e : rsrcs.entrySet()) { LocalResourcesTracker tracker = getLocalResourcesTracker(e.getKey(), c.getUser(), c.getContainerId().getApplicationAttemptId() .getApplicationId()); for (LocalResourceRequest req : e.getValue()) { + XTrace.set(start_context); tracker.handle(new ResourceReleaseEvent(req, c.getContainerId())); + end_context.add(XTrace.get()); } } + for (Context ctx : end_context) + XTrace.set(ctx); String locId = ConverterUtils.toString(c.getContainerId()); localizerTracker.cleanupPrivLocalizers(locId); @@ -571,6 +590,7 @@ public void serviceStop() throws Exception { @Override public void handle(LocalizerEvent event) { + event.joinContext(); String locId = event.getLocalizerId(); switch (event.getType()) { case REQUEST_RESOURCE_LOCALIZATION: @@ -700,11 +720,14 @@ public void run() { try { // TODO shutdown, better error handling esp. DU while (!Thread.currentThread().isInterrupted()) { + XTrace.stop(); try { Future completed = queue.take(); LocalizerResourceRequestEvent assoc = pending.remove(completed); try { Path local = completed.get(); +// // TODO: XTrace deal with this +// XTraceContext.joinObject(local); if (null == assoc) { LOG.error("Localized unkonwn resource to " + completed); // TODO delete @@ -775,6 +798,7 @@ public void addResource(LocalizerResourceRequestEvent request) { */ private LocalResource findNextResource() { // TODO: Synchronization + XTrace.stop(); for (Iterator i = pending.iterator(); i.hasNext();) { LocalizerResourceRequestEvent evt = i.next(); @@ -793,6 +817,7 @@ private LocalResource findNextResource() { */ if (nRsrc.tryAcquire()) { if (nRsrc.getState().equals(ResourceState.DOWNLOADING)) { + evt.joinContext(); LocalResourceRequest nextRsrc = nRsrc.getRequest(); LocalResource next = recordFactory.newRecordInstance(LocalResource.class); @@ -803,6 +828,7 @@ private LocalResource findNextResource() { next.setVisibility(evt.getVisibility()); next.setPattern(evt.getPattern()); scheduled.put(nextRsrc, evt); + XTrace.stop(); return next; } else { // Need to release acquired lock @@ -857,6 +883,7 @@ LocalizerHeartbeatResponse update( */ for (LocalResourceStatus stat : remoteResourceStatuses) { + stat.joinContext(); LocalResource rsrc = stat.getResource(); LocalResourceRequest req = null; try { @@ -870,6 +897,7 @@ LocalizerHeartbeatResponse update( LOG.error("Unknown resource reported: " + req); continue; } + assoc.joinContext(); switch (stat.getStatus()) { case FETCH_SUCCESS: // notify resource @@ -931,6 +959,7 @@ LocalizerHeartbeatResponse update( req, stat.getException().getMessage())); break; } + XTrace.stop(); } response.setResourceSpecs(rsrcs); return response; @@ -1047,6 +1076,7 @@ public CacheCleanup(Dispatcher dispatcher) { @Override @SuppressWarnings("unchecked") // dispatcher not typed public void run() { + xtrace.log("Cache Cleanup Thread started"); dispatcher.getEventHandler().handle( new LocalizationEvent(LocalizationEventType.CACHE_CLEANUP)); } 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/logaggregation/LogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java index efe8984694e4..7f7dda2bef16 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java @@ -406,6 +406,7 @@ private void stopApp(ApplicationId appId) { @Override public void handle(LogHandlerEvent event) { + event.joinContext(); switch (event.getType()) { case APPLICATION_STARTED: LogHandlerAppStartedEvent appStartEvent = 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/loghandler/NonAggregatingLogHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java index 06b654ae0640..d875268b4820 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java @@ -98,6 +98,7 @@ protected void serviceStop() throws Exception { @SuppressWarnings("unchecked") @Override public void handle(LogHandlerEvent event) { + event.joinContext(); switch (event.getType()) { case APPLICATION_STARTED: LogHandlerAppStartedEvent appStartedEvent = 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/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java index b681b34aad5e..41fdb3cd2bfd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java @@ -37,16 +37,18 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent; -import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin; +import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import com.google.common.base.Preconditions; +import edu.brown.cs.systems.xtrace.XTrace; + public class ContainersMonitorImpl extends AbstractService implements ContainersMonitor { - final static Log LOG = LogFactory - .getLog(ContainersMonitorImpl.class); + final static XTrace.Logger xtrace = XTrace.getLogger(ContainersMonitorImpl.class); + final static Log LOG = LogFactory.getLog(ContainersMonitorImpl.class); private long monitoringInterval; private MonitoringThread monitoringThread; @@ -308,13 +310,14 @@ boolean isProcessTreeOverLimit(ResourceCalculatorProcessTree pTree, } private class MonitoringThread extends Thread { + public MonitoringThread() { super("Container Monitor"); } @Override public void run() { - + xtrace.log("Container Monitor Thread started"); while (true) { // Print the processTrees for debugging. @@ -525,6 +528,7 @@ public boolean isVmemCheckEnabled() { @Override public void handle(ContainersMonitorEvent monitoringEvent) { + monitoringEvent.joinContext(); if (!isEnabled()) { return; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto index 166ea5b37026..0dd66a1abdb7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto @@ -35,6 +35,7 @@ message LocalResourceStatusProto { optional URLProto localPath = 3; optional int64 localSize = 4; optional SerializedExceptionProto exception = 5; + optional bytes xtrace = 6; // X-Trace metadata } message LocalizerStatusProto { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java index 1eeab7c81ae0..c2c7c886a762 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java @@ -79,4 +79,9 @@ && getLocalPath().equals(other.getLocalPath())) public int hashCode() { return 4344; } + @Override + public void rememberContext(){} + @Override + public void joinContext(){}; } + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java index 14456f535a9a..17c4181bbc61 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java @@ -84,10 +84,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider; import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import edu.brown.cs.systems.xtrace.XTrace; + @SuppressWarnings("unchecked") @Private public class ApplicationMasterService extends AbstractService implements ApplicationMasterProtocol { + private static final XTrace.Logger xtrace = XTrace.getLogger(ApplicationMasterService.class); private static final Log LOG = LogFactory.getLog(ApplicationMasterService.class); private final AMLivelinessMonitor amLivelinessMonitor; private YarnScheduler rScheduler; @@ -530,6 +533,7 @@ public void registerAppAttempt(ApplicationAttemptId attemptId) { // attemptID get registered response.setResponseId(-1); LOG.info("Registering app attempt : " + attemptId); + xtrace.log("Registering app attempt", "Attempt ID", attemptId); responseMap.put(attemptId, response); rmContext.getNMTokenSecretManager().registerApplicationAttempt(attemptId); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index 1f7a8477d6ee..f70d73bb01f7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -43,10 +43,10 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; @@ -95,6 +95,8 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Records; +import edu.brown.cs.systems.xtrace.XTrace; + /** * The client interface to the Resource Manager. This module handles all the rpc @@ -104,6 +106,7 @@ public class ClientRMService extends AbstractService implements ApplicationClientProtocol { private static final ArrayList EMPTY_APPS_REPORT = new ArrayList(); + private static final XTrace.Logger xtrace = XTrace.getLogger(ClientRMService.class); private static final Log LOG = LogFactory.getLog(ClientRMService.class); final private AtomicInteger applicationCounter = new AtomicInteger(0); @@ -201,6 +204,7 @@ ApplicationId getNewApplicationId() { .newApplicationId(recordFactory, ResourceManager.clusterTimeStamp, applicationCounter.incrementAndGet()); LOG.info("Allocated new applicationId: " + applicationId.getId()); + xtrace.log("New application ID allocated", "Application ID", applicationId.getId()); return applicationId; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java index e9a043693e10..08b2420fea9f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java @@ -142,6 +142,7 @@ public int getUnusableNodes(Collection unUsableNodes) { @Override public void handle(NodesListManagerEvent event) { + event.joinContext(); RMNode eventNode = event.getNode(); switch (event.getType()) { case NODE_UNUSABLE: diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index 6439df1c2251..6338e6e323e7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -54,12 +54,15 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import edu.brown.cs.systems.xtrace.XTrace; + /** * This class manages the list of applications for the resource manager. */ public class RMAppManager implements EventHandler, Recoverable { + private static final XTrace.Logger xtrace = XTrace.getLogger(RMAppManager.class); private static final Log LOG = LogFactory.getLog(RMAppManager.class); private int completedAppsMax = YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS; @@ -241,6 +244,7 @@ protected void submitApplication( ApplicationSubmissionContext submissionContext, long submitTime, boolean isRecovered, String user) throws YarnException { ApplicationId applicationId = submissionContext.getApplicationId(); + xtrace.log("Submitting application", "Application ID", applicationId.getId()); // Validation of the ApplicationSubmissionContext needs to be completed // here. Only those fields that are dependent on RM's configuration are @@ -383,6 +387,7 @@ public void recover(RMState state) throws Exception { @Override public void handle(RMAppManagerEvent event) { + event.joinContext(); ApplicationId applicationId = event.getApplicationId(); LOG.debug("RMAppManager processing event for " + applicationId + " of type " + event.getType()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index 841f387e7d9a..284a8eaf9f38 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -67,18 +67,18 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEventType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer; +import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager; -import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.webproxy.AppReportFetcher; @@ -91,6 +91,8 @@ import com.google.common.annotations.VisibleForTesting; +import edu.brown.cs.systems.xtrace.XTrace; + /** * The ResourceManager is the main class that is a set of components. * "I am the ResourceManager. All your resources belong to us..." @@ -401,12 +403,14 @@ public void run() { SchedulerEvent event; while (!stopped && !Thread.currentThread().isInterrupted()) { + XTrace.stop(); try { event = eventQueue.take(); } catch (InterruptedException e) { LOG.error("Returning, interrupted : " + e); return; // TODO: Kill RM. } + event.joinContext(); try { scheduler.handle(event); @@ -444,6 +448,7 @@ protected void serviceStop() throws Exception { @Override public void handle(SchedulerEvent event) { + event.joinContext(); try { int qSize = eventQueue.size(); if (qSize !=0 && qSize %1000 == 0) { @@ -454,6 +459,7 @@ public void handle(SchedulerEvent event) { LOG.info("Very low remaining capacity on scheduler event queue: " + remCapacity); } + event.rememberContext(); this.eventQueue.put(event); } catch (InterruptedException e) { throw new YarnRuntimeException(e); @@ -473,6 +479,7 @@ public ApplicationEventDispatcher(RMContext rmContext) { @Override public void handle(RMAppEvent event) { + event.joinContext(); ApplicationId appID = event.getApplicationId(); RMApp rmApp = this.rmContext.getRMApps().get(appID); if (rmApp != null) { @@ -528,6 +535,7 @@ public ApplicationAttemptEventDispatcher(RMContext rmContext) { @Override public void handle(RMAppAttemptEvent event) { + event.joinContext(); ApplicationAttemptId appAttemptID = event.getApplicationAttemptId(); ApplicationId appAttemptId = appAttemptID.getApplicationId(); RMApp rmApp = this.rmContext.getRMApps().get(appAttemptId); @@ -557,6 +565,7 @@ public NodeEventDispatcher(RMContext rmContext) { @Override public void handle(RMNodeEvent event) { + event.joinContext(); NodeId nodeId = event.getNodeId(); RMNode node = this.rmContext.getRMNodes().get(nodeId); if (node != null) { @@ -815,6 +824,7 @@ public void recover(RMState state) throws Exception { } public static void main(String argv[]) { + System.out.println("I am ResourceManager, king of kings. Look on my works ye mighty, and despair."); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); StringUtils.startupShutdownMessage(ResourceManager.class, argv, LOG); try { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java index a1c1a40d1bfa..5b814a722b52 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java @@ -34,8 +34,8 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.SecretManager.InvalidToken; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; @@ -63,11 +63,14 @@ import com.google.common.annotations.VisibleForTesting; +import edu.brown.cs.systems.xtrace.XTrace; + /** * The launch of the AM itself. */ public class AMLauncher implements Runnable { + private static final XTrace.Logger xtrace = XTrace.getLogger(AMLauncher.class); private static final Log LOG = LogFactory.getLog(AMLauncher.class); private ContainerManagementProtocol containerMgrProxy; @@ -80,7 +83,7 @@ public class AMLauncher implements Runnable { @SuppressWarnings("rawtypes") private final EventHandler handler; - + public AMLauncher(RMContext rmContext, RMAppAttempt application, AMLauncherEventType eventType, Configuration conf) { this.application = application; @@ -103,7 +106,9 @@ private void launch() throws IOException, YarnException { ApplicationSubmissionContext applicationContext = application.getSubmissionContext(); LOG.info("Setting up container " + masterContainer - + " for AM " + application.getAppAttemptId()); + + " for AM " + application.getAppAttemptId()); + xtrace.log("Setting up container for application master", + "Container", application.getMasterContainer(), "Application Master ID", application.getAppAttemptId()); ContainerLaunchContext launchContext = createAMContainerLaunchContext(applicationContext, masterContainerID); @@ -245,7 +250,8 @@ public void run() { switch (eventType) { case LAUNCH: try { - LOG.info("Launching master" + application.getAppAttemptId()); + LOG.info("Launching master " + application.getAppAttemptId()); + xtrace.log("Launching application master", "Application Master ID", application.getAppAttemptId()); launch(); handler.handle(new RMAppAttemptEvent(application.getAppAttemptId(), RMAppAttemptEventType.LAUNCHED)); @@ -260,6 +266,7 @@ public void run() { case CLEANUP: try { LOG.info("Cleaning master " + application.getAppAttemptId()); + xtrace.log("Cleaning application master", "Application Master ID", application.getAppAttemptId()); cleanup(); } catch(IOException ie) { LOG.info("Error cleaning master ", ie); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java index af02b198cf76..8dfa5e3ebc05 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java @@ -109,6 +109,7 @@ private void cleanup(RMAppAttempt application) { @Override public synchronized void handle(AMLauncherEvent appEvent) { + appEvent.joinContext(); AMLauncherEventType event = appEvent.getType(); RMAppAttempt application = appEvent.getAppAttempt(); switch (event) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java index a1c234cad236..24d2621fa651 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEvent.java @@ -21,9 +21,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.event.AbstractEvent; +import edu.brown.cs.systems.xtrace.Context; + public class RMAppEvent extends AbstractEvent{ private final ApplicationId appId; + public Context last_transition; public RMAppEvent(ApplicationId appId, RMAppEventType type) { super(type); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java index a11b05eedc97..1abde9dafbf3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java @@ -118,7 +118,7 @@ public class RMAppImpl implements RMApp, Recoverable { = new StateMachineFactory(RMAppState.NEW) + RMAppEvent>(RMAppState.NEW, StateMachineFactory.Trace.KEEPALIVE) // Transitions from NEW state @@ -169,15 +169,15 @@ RMAppEventType.KILL, new KillAppAndAttemptTransition()) .addTransition(RMAppState.RUNNING, RMAppState.RUNNING, RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition()) .addTransition(RMAppState.RUNNING, RMAppState.FINISHING, - RMAppEventType.ATTEMPT_FINISHING, new RMAppFinishingTransition()) + RMAppEventType.ATTEMPT_FINISHING, new RMAppFinishingTransition(), StateMachineFactory.Trace.ALWAYS) .addTransition(RMAppState.RUNNING, RMAppState.FINISHED, - RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION) + RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION, StateMachineFactory.Trace.ALWAYS) .addTransition(RMAppState.RUNNING, EnumSet.of(RMAppState.SUBMITTED, RMAppState.FAILED), RMAppEventType.ATTEMPT_FAILED, - new AttemptFailedTransition(RMAppState.SUBMITTED)) + new AttemptFailedTransition(RMAppState.SUBMITTED), StateMachineFactory.Trace.ALWAYS) .addTransition(RMAppState.RUNNING, RMAppState.KILLED, - RMAppEventType.KILL, new KillAppAndAttemptTransition()) + RMAppEventType.KILL, new KillAppAndAttemptTransition(), StateMachineFactory.Trace.ALWAYS) // Transitions from FINISHING state .addTransition(RMAppState.FINISHING, RMAppState.FINISHED, @@ -568,6 +568,7 @@ public int getMaxAppAttempts() { @Override public void handle(RMAppEvent event) { + event.joinContext(); this.writeLock.lock(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java index 4a877cadae78..eeec1eb4f5f6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java @@ -99,9 +99,12 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.resource.Resources; +import edu.brown.cs.systems.xtrace.XTrace; + @SuppressWarnings({"unchecked", "rawtypes"}) public class RMAppAttemptImpl implements RMAppAttempt, Recoverable { + private static final XTrace.Logger xtrace = XTrace.getLogger(RMAppAttemptImpl.class); private static final Log LOG = LogFactory.getLog(RMAppAttemptImpl.class); private static final RecordFactory recordFactory = RecordFactoryProvider @@ -162,7 +165,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable { stateMachineFactory = new StateMachineFactory(RMAppAttemptState.NEW) + RMAppAttemptEvent>(RMAppAttemptState.NEW, StateMachineFactory.Trace.NEVER) // Transitions from NEW State .addTransition(RMAppAttemptState.NEW, RMAppAttemptState.SUBMITTED, @@ -587,6 +590,7 @@ private void setMasterContainer(Container container) { @Override public void handle(RMAppAttemptEvent event) { + event.joinContext(); this.writeLock.lock(); @@ -786,6 +790,8 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt, BuilderUtils.newResourceRequest( AM_CONTAINER_PRIORITY, ResourceRequest.ANY, appAttempt .getSubmissionContext().getResource(), 1); + xtrace.log("Requesting container for ApplicationMaster"); + request.rememberContext(); // SchedulerUtils.validateResourceRequests is not necessary because // AM resource has been checked when submission diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java index d44fd3f32c2e..f92cd6e6250d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java @@ -50,7 +50,7 @@ public class RMContainerImpl implements RMContainer { RMContainerEventType, RMContainerEvent> stateMachineFactory = new StateMachineFactory( - RMContainerState.NEW) + RMContainerState.NEW, StateMachineFactory.Trace.KEEPALIVE) // Transitions from NEW state .addTransition(RMContainerState.NEW, RMContainerState.ALLOCATED, @@ -205,6 +205,8 @@ public String toString() { @Override public void handle(RMContainerEvent event) { + // X-Trace: this handle method is only ever called synchronously (it seems) + event.joinContext(); LOG.debug("Processing " + event.getContainerId() + " of type " + event.getType()); try { writeLock.lock(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java index 70507f6ad2b0..33047786575d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java @@ -63,6 +63,8 @@ import com.google.common.annotations.VisibleForTesting; +import edu.brown.cs.systems.xtrace.XTrace; + /** * This class is used to keep track of all the applications/containers * running on a node. @@ -73,6 +75,7 @@ @SuppressWarnings("unchecked") public class RMNodeImpl implements RMNode, EventHandler { + private static final XTrace.Logger xtrace = XTrace.getLogger(RMNodeImpl.class); private static final Log LOG = LogFactory.getLog(RMNodeImpl.class); private static final RecordFactory recordFactory = RecordFactoryProvider @@ -118,7 +121,7 @@ public class RMNodeImpl implements RMNode, EventHandler { = new StateMachineFactory(NodeState.NEW) + RMNodeEvent>(NodeState.NEW, StateMachineFactory.Trace.NEVER) //Transitions from NEW state .addTransition(NodeState.NEW, NodeState.RUNNING, @@ -350,6 +353,7 @@ public NodeHeartbeatResponse getLastNodeHeartBeatResponse() { } public void handle(RMNodeEvent event) { + event.joinContext(); LOG.debug("Processing " + event.getNodeId() + " of type " + event.getType()); try { writeLock.lock(); @@ -484,8 +488,9 @@ public static class CleanUpContainerTransition implements @Override public void transition(RMNodeImpl rmNode, RMNodeEvent event) { - rmNode.containersToClean.add((( - RMNodeCleanContainerEvent) event).getContainerId()); + ContainerId id = ((RMNodeCleanContainerEvent) event).getContainerId(); + id.rememberContext(); + rmNode.containersToClean.add(id); } } @@ -557,12 +562,16 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { for (ContainerStatus remoteContainer : statusEvent.getContainers()) { ContainerId containerId = remoteContainer.getContainerId(); + XTrace.stop(); + containerId.joinContext(); + // Don't bother with containers already scheduled for cleanup, or for // applications already killed. The scheduler doens't need to know any // more about this container if (rmNode.containersToClean.contains(containerId)) { LOG.info("Container " + containerId + " already scheduled for " + "cleanup, no further processing"); + xtrace.log("Container already scheduled for cleanup", "Container ID", containerId); continue; } if (rmNode.finishedApplications.contains(containerId @@ -570,6 +579,7 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { LOG.info("Container " + containerId + " belongs to an application that is already killed," + " no further processing"); + xtrace.log("Container belongs to an application that is already killed", "Container ID", containerId); continue; } @@ -580,6 +590,7 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { rmNode.justLaunchedContainers.put(containerId, remoteContainer); newlyLaunchedContainers.add(remoteContainer); } + rmNode.justLaunchedContainers.get(containerId).getContainerId().rememberContext(); } else { // A finished container rmNode.justLaunchedContainers.remove(containerId); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index 29c4d4b9de31..1fb444c9b0fb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -80,6 +80,8 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import edu.brown.cs.systems.xtrace.XTrace; + @LimitedPrivate("yarn") @Evolving @SuppressWarnings("unchecked") @@ -87,6 +89,7 @@ public class CapacityScheduler implements PreemptableResourceScheduler, CapacitySchedulerContext, Configurable { + private static final XTrace.Logger xtrace = XTrace.getLogger(CapacityScheduler.class); private static final Log LOG = LogFactory.getLog(CapacityScheduler.class); private CSQueue root; @@ -637,9 +640,13 @@ private synchronized void nodeUpdate(RMNode nm) { // Process completed containers for (ContainerStatus completedContainer : completedContainers) { ContainerId containerId = completedContainer.getContainerId(); + XTrace.stop(); + containerId.joinContext(); LOG.debug("Container FINISHED: " + containerId); + xtrace.log("Container Finished", "Container ID",containerId); completedContainer(getRMContainer(containerId), completedContainer, RMContainerEventType.FINISHED); + XTrace.stop(); } // Now node data structures are upto date and ready for scheduling. @@ -691,6 +698,9 @@ private synchronized void nodeUpdate(RMNode nm) { } private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { + XTrace.stop(); + containerId.joinContext(); + // Get the application for the finished container ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); FiCaSchedulerApp application = getApplication(applicationAttemptId); @@ -704,10 +714,12 @@ private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode } application.containerLaunchedOnNode(containerId, node.getNodeID()); + XTrace.stop(); } @Override public void handle(SchedulerEvent event) { + event.joinContext(); switch(event.getType()) { case NODE_ADDED: { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index c2c5d27576dd..48c8c4756eb2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -67,6 +67,8 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import edu.brown.cs.systems.xtrace.XTrace; + @Private @Unstable public class LeafQueue implements CSQueue { @@ -1301,6 +1303,12 @@ Token createContainerToken( private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, ResourceRequest request, NodeType type, RMContainer rmContainer) { + + XTrace.stop(); + request.joinContext(); + + try { // xtrace try + if (LOG.isDebugEnabled()) { LOG.debug("assignContainers: node=" + node.getHostName() + " application=" + application.getApplicationId().getId() @@ -1382,6 +1390,10 @@ private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode nod return request.getCapability(); } + + } finally { // xtrace finally + XTrace.stop(); + } } private void reserve(FiCaSchedulerApp application, Priority priority, @@ -1617,4 +1629,4 @@ public Resource getTotalResourcePending() { return ret; } -} +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index a261dbfd5a27..97b0b8e1c9e8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -60,12 +60,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; -import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; +import org.apache.hadoop.yarn.util.resource.Resources; import com.google.common.collect.HashMultiset; import com.google.common.collect.Multiset; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; + /** * Represents an Application from the viewpoint of the scheduler. * Each running Application in the RM corresponds to one instance @@ -308,10 +311,13 @@ synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node, synchronized public List pullNewlyAllocatedContainers() { List returnContainerList = new ArrayList( newlyAllocatedContainers.size()); + Context start_context = XTrace.get(); for (RMContainer rmContainer : newlyAllocatedContainers) { rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(), RMContainerEventType.ACQUIRED)); returnContainerList.add(rmContainer.getContainer()); + rmContainer.getContainerId().rememberContext(); + XTrace.set(start_context); } newlyAllocatedContainers.clear(); return returnContainerList; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java index 81fbc16f16a7..ad6e4b9aba82 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java @@ -35,10 +35,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -51,6 +49,7 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; @@ -91,6 +90,8 @@ import com.google.common.annotations.VisibleForTesting; +import edu.brown.cs.systems.xtrace.XTrace; + /** * A scheduler that schedules resources between a set of queues. The scheduler * keeps track of the resources used by each queue, and attempts to maintain @@ -123,6 +124,7 @@ public class FairScheduler implements ResourceScheduler { private QueueManager queueMgr; private Clock clock; + private static final XTrace.Logger xtrace = XTrace.getLogger(); private static final Log LOG = LogFactory.getLog(FairScheduler.class); private static final ResourceCalculator RESOURCE_CALCULATOR = @@ -852,6 +854,9 @@ public Allocation allocate(ApplicationAttemptId appAttemptId, * Process a container which has launched on a node, as reported by the node. */ private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) { + XTrace.stop(); + containerId.joinContext(); + // Get the application for the finished container ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); FSSchedulerApp application = applications.get(applicationAttemptId); @@ -863,6 +868,7 @@ private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode no } application.containerLaunchedOnNode(containerId, node.getNodeID()); + XTrace.stop(); } /** @@ -890,9 +896,13 @@ private synchronized void nodeUpdate(RMNode nm) { // Process completed containers for (ContainerStatus completedContainer : completedContainers) { ContainerId containerId = completedContainer.getContainerId(); + XTrace.stop(); + containerId.joinContext(); LOG.debug("Container FINISHED: " + containerId); + xtrace.log("Container Finished", "Container ID", containerId); completedContainer(getRMContainer(containerId), completedContainer, RMContainerEventType.FINISHED); + XTrace.stop(); } // Assign new containers... diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index d971f3b4496e..c65afb572336 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -36,12 +36,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; 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.Token; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.QueueACL; @@ -50,7 +48,9 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; @@ -90,11 +90,15 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import edu.brown.cs.systems.xtrace.Context; +import edu.brown.cs.systems.xtrace.XTrace; + @LimitedPrivate("yarn") @Evolving @SuppressWarnings("unchecked") public class FifoScheduler implements ResourceScheduler, Configurable { + private static final XTrace.Logger xtrace = XTrace.getLogger(FifoScheduler.class); private static final Log LOG = LogFactory.getLog(FifoScheduler.class); private static final RecordFactory recordFactory = @@ -258,7 +262,10 @@ public Allocation allocate( clusterResource, minimumAllocation, maximumAllocation); // Release containers + Context start_context = XTrace.get(); for (ContainerId releasedContainer : release) { + XTrace.set(start_context); + releasedContainer.joinContext(); RMContainer rmContainer = getRMContainer(releasedContainer); if (rmContainer == null) { RMAuditLogger.logFailure(application.getUser(), @@ -273,6 +280,7 @@ public Allocation allocate( SchedulerUtils.RELEASED_CONTAINER), RMContainerEventType.RELEASED); } + XTrace.set(start_context); synchronized (application) { @@ -553,6 +561,9 @@ private int assignOffSwitchContainers(FiCaSchedulerNode node, private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, int assignableContainers, ResourceRequest request, NodeType type) { + XTrace.stop(); + request.joinContext(); + LOG.debug("assignContainers:" + " node=" + node.getRMNode().getNodeAddress() + " application=" + application.getApplicationId().getId() + @@ -607,6 +618,7 @@ private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application } } + XTrace.stop(); return assignedContainers; } @@ -629,9 +641,13 @@ private synchronized void nodeUpdate(RMNode rmNode) { // Process completed containers for (ContainerStatus completedContainer : completedContainers) { ContainerId containerId = completedContainer.getContainerId(); + XTrace.stop(); + containerId.joinContext(); LOG.debug("Container FINISHED: " + containerId); + xtrace.log("Container Finished", "Container ID", containerId); containerCompleted(getRMContainer(containerId), completedContainer, RMContainerEventType.FINISHED); + XTrace.stop(); } if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource, @@ -651,6 +667,7 @@ private synchronized void nodeUpdate(RMNode rmNode) { @Override public void handle(SchedulerEvent event) { + event.joinContext(); switch(event.getType()) { case NODE_ADDED: { @@ -708,6 +725,9 @@ public void handle(SchedulerEvent event) { } private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { + XTrace.stop(); + containerId.joinContext(); + // Get the application for the finished container ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); FiCaSchedulerApp application = getApplication(applicationAttemptId); @@ -723,6 +743,7 @@ private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode } application.containerLaunchedOnNode(containerId, node.getNodeID()); + XTrace.stop(); } @Lock(FifoScheduler.class) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java index 2c9d67845a4f..d1c541a32db0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java @@ -313,4 +313,4 @@ synchronized public GetContainerStatusesResponse getContainerStatuses( nodeStatus.setNodeHealthStatus(nodeHealthStatus); return nodeStatus; } -} +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 6dfd5d1f4c16..be669e538673 100644 --- a/pom.xml +++ b/pom.xml @@ -23,6 +23,34 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs Apache Hadoop Main pom + + + org.aspectj + aspectjrt + 1.7.3 + + + edu.brown.cs.systems + xtrace + 3.0 + + + edu.brown.cs.systems + xtrace-resource-tracing + 1.0 + + + edu.brown.cs.systems + resourcethrottling + 1.0 + + + com.google.guava + guava + 17.0 + + + @@ -161,6 +189,34 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs + + org.codehaus.mojo + aspectj-maven-plugin + 1.5 + + + + edu.brown.cs.systems + xtrace-resource-tracing + + + + + + + compile + + + 1.6 + 1.6 + 1.6 + synchronization + true + true + + + + org.apache.maven.plugins maven-enforcer-plugin From 78ed44ed81f365dbacdba3a3810e246b2a1adb6d Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 10 Dec 2014 14:40:05 -0500 Subject: [PATCH 103/112] Temporary random shuffle of datanodes --- .../blockmanagement/DatanodeManager.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index ff93d73d5385..2c995ffa1148 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -29,8 +29,10 @@ import java.util.Iterator; import java.util.List; import java.util.NavigableMap; +import java.util.Random; import java.util.TreeMap; +import org.apache.commons.lang3.RandomUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.HadoopIllegalArgumentException; @@ -343,12 +345,28 @@ public void sortLocatedBlocks(final String targethost, DFSUtil.DECOM_COMPARATOR; for (LocatedBlock b : locatedblocks) { + // temporary hack by jon: randomly permute the locations + shuffleArray(b.getLocations()); networktopology.pseudoSortByDistance(client, b.getLocations()); // Move decommissioned/stale datanodes to the bottom Arrays.sort(b.getLocations(), comparator); } } + private static final Random r = new Random(); + // Implementing Fisher–Yates shuffle + private static void shuffleArray(T[] ar) + { + for (int i = ar.length - 1; i > 0; i--) + { + int index = r.nextInt(i + 1); + // Simple swap + T a = ar[index]; + ar[index] = ar[i]; + ar[i] = a; + } + } + CyclicIteration getDatanodeCyclicIteration( final String firstkey) { return new CyclicIteration( From a9f3d0f74287fc1a652ca0c7ae11fc3dd3d0c01d Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Wed, 10 Dec 2014 14:45:08 -0500 Subject: [PATCH 104/112] Remove non-utf8 character --- .../hadoop/hdfs/server/blockmanagement/DatanodeManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index 2c995ffa1148..b78b845a2d99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -354,7 +354,7 @@ public void sortLocatedBlocks(final String targethost, } private static final Random r = new Random(); - // Implementing Fisher–Yates shuffle + // Implementing Fisher Yates shuffle private static void shuffleArray(T[] ar) { for (int i = ar.length - 1; i > 0; i--) From a572a0a34e0a58b8f2223d562aaed2b2ea1e5240 Mon Sep 17 00:00:00 2001 From: rroelke Date: Mon, 22 Dec 2014 12:13:00 -0500 Subject: [PATCH 105/112] added mvn dependency on PivotTracing from retro --- pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pom.xml b/pom.xml index be669e538673..44e323ca20ff 100644 --- a/pom.xml +++ b/pom.xml @@ -44,6 +44,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs resourcethrottling 1.0 + + edu.brown.cs.systems + pivot-tracing + 1.0 + com.google.guava guava From 3e7f5740464f0d4c7c4633a775e53b55a699bcea Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Mon, 22 Dec 2014 12:32:47 -0500 Subject: [PATCH 106/112] Add pivot tracing to aspectj libraries --- pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pom.xml b/pom.xml index 44e323ca20ff..b00924359b56 100644 --- a/pom.xml +++ b/pom.xml @@ -204,6 +204,10 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs edu.brown.cs.systems xtrace-resource-tracing + + edu.brown.cs.systems + pivot-tracing + From 0172dbcd1b997525f96539de3121dab54a6db63f Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Sun, 8 Mar 2015 03:12:42 -0400 Subject: [PATCH 107/112] Add pivot tracing init to HDFS main methods --- .../java/org/apache/hadoop/hdfs/DistributedFileSystem.java | 3 +++ .../org/apache/hadoop/hdfs/server/datanode/DataNode.java | 2 ++ .../org/apache/hadoop/hdfs/server/namenode/NameNode.java | 5 +++++ 3 files changed, 10 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index a32c142ff1b7..a563cd82fbfa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -80,6 +80,8 @@ import com.google.common.annotations.VisibleForTesting; +import edu.brown.cs.systems.pivottracing.PivotTracingPubSub; + /**************************************************************** * Implementation of the abstract FileSystem for the DFS system. @@ -98,6 +100,7 @@ public class DistributedFileSystem extends FileSystem { static{ HdfsConfiguration.init(); + PivotTracingPubSub.initialize(); } public DistributedFileSystem() { 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 11d49cae73ee..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,7 @@ 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; @@ -1938,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); } 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); } From 32f8d5f5cc27c1fe4f4b3a9cd36de7b45a48777c Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Sun, 8 Mar 2015 15:57:35 -0400 Subject: [PATCH 108/112] Add extra room in the packet header for pivot tracing baggage --- .../apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java index e7aadeb8fbc2..33b15563ed5e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java @@ -64,7 +64,7 @@ public class PacketHeader { .setDataLen(0) .setSyncBlock(false) .setXtrace(ByteString.copyFrom(XTrace.XTRACE_BYTES_EXAMPLE)) - .build().getSerializedSize(); + .build().getSerializedSize() + 1024; } public static final int PKT_LENGTHS_LEN = Ints.BYTES + Shorts.BYTES; From 04109334938671aa64f69ccdd07b88b55a82a136 Mon Sep 17 00:00:00 2001 From: jon mace Date: Mon, 16 Mar 2015 17:00:52 -0400 Subject: [PATCH 109/112] Add agentlib to HDFS command --- hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs index f8c6bb7573ad..0da300153005 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs @@ -155,7 +155,7 @@ fi export CLASSPATH=$CLASSPATH -HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,NullAppender}" +HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,NullAppender} -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=0" # Check to see if we should start a secure datanode if [ "$starting_secure_dn" = "true" ]; then From 2a72991ffcdb0325d437928a74026c3032cb7c19 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 17 Mar 2015 15:22:15 -0400 Subject: [PATCH 110/112] Add command line option to set mapreduce tenant class --- .../main/java/org/apache/hadoop/util/ProgramDriver.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java index 956fb58b673e..0e32f28c2efa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java @@ -139,7 +139,13 @@ public int run(String[] args) } XTrace.startTask(true); - XTrace.setTenantClass(55); + XTrace.startTask(true); + int tenant = 55; + try { + ConfigFactory.load().getInt("mapreduce.tenant"); + } catch (Exception e) { + } + XTrace.setTenantClass(tenant); XTrace.getLogger("ProgramDriver").log("Executing example program", "ProgramName", args[0]); // Remove the leading argument and call main From 9c4f15be04aaacef7ab013fbe5569ca1d1ccc3b7 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 17 Mar 2015 15:24:05 -0400 Subject: [PATCH 111/112] Fix partial accidental commit --- .../src/main/java/org/apache/hadoop/util/ProgramDriver.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java index 0e32f28c2efa..e62a1c3c68d0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java @@ -26,6 +26,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import com.typesafe.config.ConfigFactory; + import edu.brown.cs.systems.xtrace.XTrace; /** A driver that is used to run programs added to it @@ -138,7 +140,6 @@ public int run(String[] args) return -1; } - XTrace.startTask(true); XTrace.startTask(true); int tenant = 55; try { From fb0c79821765c7ac8dbaea21a03983aca31f2397 Mon Sep 17 00:00:00 2001 From: Jonathan Mace Date: Tue, 17 Mar 2015 17:07:26 -0400 Subject: [PATCH 112/112] Why am i such an idiot --- .../src/main/java/org/apache/hadoop/util/ProgramDriver.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java index e62a1c3c68d0..736e48981193 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java @@ -143,7 +143,7 @@ public int run(String[] args) XTrace.startTask(true); int tenant = 55; try { - ConfigFactory.load().getInt("mapreduce.tenant"); + tenant = ConfigFactory.load().getInt("mapreduce.tenant"); } catch (Exception e) { } XTrace.setTenantClass(tenant);