HDFS-9080. Update htrace version to 4.0.1 (cmccabe)

(cherry picked from commit 892ade689f)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
        hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
This commit is contained in:
Colin Patrick Mccabe 2015-09-26 22:05:51 -07:00
parent 03e370e306
commit 3b9d32627f
63 changed files with 876 additions and 782 deletions

View File

@ -246,7 +246,7 @@
<dependency> <dependency>
<groupId>org.apache.htrace</groupId> <groupId>org.apache.htrace</groupId>
<artifactId>htrace-core</artifactId> <artifactId>htrace-core4</artifactId>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.zookeeper</groupId> <groupId>org.apache.zookeeper</groupId>

View File

@ -295,4 +295,7 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
public static final String NFS_EXPORTS_ALLOWED_HOSTS_SEPARATOR = ";"; public static final String NFS_EXPORTS_ALLOWED_HOSTS_SEPARATOR = ";";
public static final String NFS_EXPORTS_ALLOWED_HOSTS_KEY = "nfs.exports.allowed.hosts"; public static final String NFS_EXPORTS_ALLOWED_HOSTS_KEY = "nfs.exports.allowed.hosts";
public static final String NFS_EXPORTS_ALLOWED_HOSTS_KEY_DEFAULT = "* rw"; public static final String NFS_EXPORTS_ALLOWED_HOSTS_KEY_DEFAULT = "* rw";
// HDFS client HTrace configuration.
public static final String FS_CLIENT_HTRACE_PREFIX = "fs.client.htrace.";
} }

View File

@ -21,8 +21,8 @@ package org.apache.hadoop.fs;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.htrace.NullScope; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceScope; import org.apache.htrace.core.Tracer;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
@ -43,6 +43,8 @@ abstract public class FSOutputSummer extends OutputStream {
private byte checksum[]; private byte checksum[];
// The number of valid bytes in the buffer. // The number of valid bytes in the buffer.
private int count; private int count;
// The HTrace tracer to use
private Tracer tracer;
// We want this value to be a multiple of 3 because the native code checksums // We want this value to be a multiple of 3 because the native code checksums
// 3 chunks simultaneously. The chosen value of 9 strikes a balance between // 3 chunks simultaneously. The chosen value of 9 strikes a balance between
@ -197,7 +199,7 @@ abstract public class FSOutputSummer extends OutputStream {
} }
protected TraceScope createWriteTraceScope() { protected TraceScope createWriteTraceScope() {
return NullScope.INSTANCE; return null;
} }
/** Generate checksums for the given data chunks and output chunks & checksums /** Generate checksums for the given data chunks and output chunks & checksums
@ -215,7 +217,9 @@ abstract public class FSOutputSummer extends OutputStream {
getChecksumSize()); getChecksumSize());
} }
} finally { } finally {
scope.close(); if (scope != null) {
scope.close();
}
} }
} }

View File

@ -61,6 +61,7 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.ShutdownHookManager;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.htrace.core.Tracer;
/** /**
* The FileContext class provides an interface to the application writer for * The FileContext class provides an interface to the application writer for
@ -232,12 +233,14 @@ public class FileContext {
private final Configuration conf; private final Configuration conf;
private final UserGroupInformation ugi; private final UserGroupInformation ugi;
final boolean resolveSymlinks; final boolean resolveSymlinks;
private final Tracer tracer;
private FileContext(final AbstractFileSystem defFs, private FileContext(final AbstractFileSystem defFs,
final FsPermission theUmask, final Configuration aConf) { final FsPermission theUmask, final Configuration aConf) {
defaultFS = defFs; defaultFS = defFs;
umask = FsPermission.getUMask(aConf); umask = FsPermission.getUMask(aConf);
conf = aConf; conf = aConf;
tracer = FsTracer.get(aConf);
try { try {
ugi = UserGroupInformation.getCurrentUser(); ugi = UserGroupInformation.getCurrentUser();
} catch (IOException e) { } catch (IOException e) {
@ -2731,4 +2734,8 @@ public class FileContext {
throws IOException { throws IOException {
return defaultFS.getAllStoragePolicies(); return defaultFS.getAllStoragePolicies();
} }
Tracer getTracer() {
return tracer;
}
} }

View File

@ -67,9 +67,8 @@ import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.Span; import org.apache.htrace.core.Tracer;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceScope;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -129,6 +128,13 @@ public abstract class FileSystem extends Configured implements Closeable {
private Set<Path> deleteOnExit = new TreeSet<Path>(); private Set<Path> deleteOnExit = new TreeSet<Path>();
boolean resolveSymlinks; boolean resolveSymlinks;
private Tracer tracer;
protected final Tracer getTracer() {
return tracer;
}
/** /**
* This method adds a file system for testing so that we can find it later. It * This method adds a file system for testing so that we can find it later. It
* is only for testing. * is only for testing.
@ -2692,14 +2698,13 @@ public abstract class FileSystem extends Configured implements Closeable {
private static FileSystem createFileSystem(URI uri, Configuration conf private static FileSystem createFileSystem(URI uri, Configuration conf
) throws IOException { ) throws IOException {
TraceScope scope = Trace.startSpan("FileSystem#createFileSystem"); Tracer tracer = FsTracer.get(conf);
Span span = scope.getSpan(); TraceScope scope = tracer.newScope("FileSystem#createFileSystem");
if (span != null) { scope.addKVAnnotation("scheme", uri.getScheme());
span.addKVAnnotation("scheme", uri.getScheme());
}
try { try {
Class<?> clazz = getFileSystemClass(uri.getScheme(), conf); Class<?> clazz = getFileSystemClass(uri.getScheme(), conf);
FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf); FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
fs.tracer = tracer;
fs.initialize(uri, conf); fs.initialize(uri, conf);
return fs; return fs;
} finally { } finally {

View File

@ -32,16 +32,13 @@ import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.shell.Command; import org.apache.hadoop.fs.shell.Command;
import org.apache.hadoop.fs.shell.CommandFactory; import org.apache.hadoop.fs.shell.CommandFactory;
import org.apache.hadoop.fs.shell.FsCommand; import org.apache.hadoop.fs.shell.FsCommand;
import org.apache.hadoop.tracing.SpanReceiverHost;
import org.apache.hadoop.tools.TableListing; import org.apache.hadoop.tools.TableListing;
import org.apache.hadoop.tracing.TraceUtils; import org.apache.hadoop.tracing.TraceUtils;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.SamplerBuilder; import org.apache.htrace.core.Tracer;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
/** Provide command line access to a FileSystem. */ /** Provide command line access to a FileSystem. */
@InterfaceAudience.Private @InterfaceAudience.Private
@ -54,13 +51,12 @@ public class FsShell extends Configured implements Tool {
private FileSystem fs; private FileSystem fs;
private Trash trash; private Trash trash;
protected CommandFactory commandFactory; protected CommandFactory commandFactory;
private Sampler traceSampler;
private final String usagePrefix = private final String usagePrefix =
"Usage: hadoop fs [generic options]"; "Usage: hadoop fs [generic options]";
private SpanReceiverHost spanReceiverHost; private Tracer tracer;
static final String SEHLL_HTRACE_PREFIX = "dfs.shell.htrace."; static final String SHELL_HTRACE_PREFIX = "fs.shell.htrace.";
/** /**
* Default ctor with no configuration. Be sure to invoke * Default ctor with no configuration. Be sure to invoke
@ -102,8 +98,9 @@ public class FsShell extends Configured implements Tool {
commandFactory.addObject(new Usage(), "-usage"); commandFactory.addObject(new Usage(), "-usage");
registerCommands(commandFactory); registerCommands(commandFactory);
} }
this.spanReceiverHost = this.tracer = new Tracer.Builder("FsShell").
SpanReceiverHost.get(getConf(), SEHLL_HTRACE_PREFIX); conf(TraceUtils.wrapHadoopConf(SHELL_HTRACE_PREFIX, getConf())).
build();
} }
protected void registerCommands(CommandFactory factory) { protected void registerCommands(CommandFactory factory) {
@ -285,8 +282,6 @@ public class FsShell extends Configured implements Tool {
public int run(String argv[]) throws Exception { public int run(String argv[]) throws Exception {
// initialize FsShell // initialize FsShell
init(); init();
traceSampler = new SamplerBuilder(TraceUtils.
wrapHadoopConf(SEHLL_HTRACE_PREFIX, getConf())).build();
int exitCode = -1; int exitCode = -1;
if (argv.length < 1) { if (argv.length < 1) {
printUsage(System.err); printUsage(System.err);
@ -298,7 +293,7 @@ public class FsShell extends Configured implements Tool {
if (instance == null) { if (instance == null) {
throw new UnknownCommandException(); throw new UnknownCommandException();
} }
TraceScope scope = Trace.startSpan(instance.getCommandName(), traceSampler); TraceScope scope = tracer.newScope(instance.getCommandName());
if (scope.getSpan() != null) { if (scope.getSpan() != null) {
String args = StringUtils.join(" ", argv); String args = StringUtils.join(" ", argv);
if (args.length() > 2048) { if (args.length() > 2048) {
@ -324,6 +319,7 @@ public class FsShell extends Configured implements Tool {
e.printStackTrace(System.err); e.printStackTrace(System.err);
} }
} }
tracer.close();
return exitCode; return exitCode;
} }
@ -350,9 +346,6 @@ public class FsShell extends Configured implements Tool {
fs.close(); fs.close();
fs = null; fs = null;
} }
if (this.spanReceiverHost != null) {
this.spanReceiverHost.closeReceivers();
}
} }
/** /**

View File

@ -0,0 +1,64 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.tracing.TraceUtils;
import org.apache.htrace.core.Tracer;
/**
* Holds the HTrace Tracer used for FileSystem operations.
*
* Ideally, this would be owned by the DFSClient, rather than global. However,
* the FileContext API may create a new DFSClient for each operation in some
* cases. Because of this, we cannot store this Tracer inside DFSClient. See
* HADOOP-6356 for details.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class FsTracer {
private static Tracer instance;
public static synchronized Tracer get(Configuration conf) {
if (instance == null) {
instance = new Tracer.Builder("FSClient").
conf(TraceUtils.wrapHadoopConf(CommonConfigurationKeys.
FS_CLIENT_HTRACE_PREFIX, conf)).
build();
}
return instance;
}
@VisibleForTesting
public static synchronized void clear() {
if (instance == null) {
return;
}
try {
instance.close();
} finally {
instance = null;
}
}
private FsTracer() {
}
}

View File

@ -28,9 +28,8 @@ import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.htrace.Span; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Unstable @InterfaceStability.Unstable
@ -41,12 +40,14 @@ class Globber {
private final FileContext fc; private final FileContext fc;
private final Path pathPattern; private final Path pathPattern;
private final PathFilter filter; private final PathFilter filter;
private final Tracer tracer;
public Globber(FileSystem fs, Path pathPattern, PathFilter filter) { public Globber(FileSystem fs, Path pathPattern, PathFilter filter) {
this.fs = fs; this.fs = fs;
this.fc = null; this.fc = null;
this.pathPattern = pathPattern; this.pathPattern = pathPattern;
this.filter = filter; this.filter = filter;
this.tracer = fs.getTracer();
} }
public Globber(FileContext fc, Path pathPattern, PathFilter filter) { public Globber(FileContext fc, Path pathPattern, PathFilter filter) {
@ -54,6 +55,7 @@ class Globber {
this.fc = fc; this.fc = fc;
this.pathPattern = pathPattern; this.pathPattern = pathPattern;
this.filter = filter; this.filter = filter;
this.tracer = fc.getTracer();
} }
private FileStatus getFileStatus(Path path) throws IOException { private FileStatus getFileStatus(Path path) throws IOException {
@ -140,11 +142,8 @@ class Globber {
} }
public FileStatus[] glob() throws IOException { public FileStatus[] glob() throws IOException {
TraceScope scope = Trace.startSpan("Globber#glob"); TraceScope scope = tracer.newScope("Globber#glob");
Span span = scope.getSpan(); scope.addKVAnnotation("pattern", pathPattern.toUri().getPath());
if (span != null) {
span.addKVAnnotation("pattern", pathPattern.toUri().getPath());
}
try { try {
return doGlob(); return doGlob();
} finally { } finally {

View File

@ -92,7 +92,8 @@ import org.apache.hadoop.util.ProtoUtil;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.htrace.Trace; import org.apache.htrace.core.Span;
import org.apache.htrace.core.Tracer;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -722,8 +723,9 @@ public class Client {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Connecting to "+server); LOG.debug("Connecting to "+server);
} }
if (Trace.isTracing()) { Span span = Tracer.getCurrentSpan();
Trace.addTimelineAnnotation("IPC client connecting to " + server); if (span != null) {
span.addTimelineAnnotation("IPC client connecting to " + server);
} }
short numRetries = 0; short numRetries = 0;
Random rand = null; Random rand = null;
@ -796,8 +798,9 @@ public class Client {
// update last activity time // update last activity time
touch(); touch();
if (Trace.isTracing()) { span = Tracer.getCurrentSpan();
Trace.addTimelineAnnotation("IPC client connected to " + server); if (span != null) {
span.addTimelineAnnotation("IPC client connected to " + server);
} }
// start the receiver thread after the socket connection has been set // start the receiver thread after the socket connection has been set

View File

@ -49,8 +49,8 @@ import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.ProtoUtil; import org.apache.hadoop.util.ProtoUtil;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceScope; import org.apache.htrace.core.Tracer;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingService; import com.google.protobuf.BlockingService;
@ -206,12 +206,13 @@ public class ProtobufRpcEngine implements RpcEngine {
+ method.getName() + "]"); + method.getName() + "]");
} }
TraceScope traceScope = null;
// if Tracing is on then start a new span for this rpc. // if Tracing is on then start a new span for this rpc.
// guard it in the if statement to make sure there isn't // guard it in the if statement to make sure there isn't
// any extra string manipulation. // any extra string manipulation.
if (Trace.isTracing()) { Tracer tracer = Tracer.curThreadTracer();
traceScope = Trace.startSpan(RpcClientUtil.methodToTraceString(method)); TraceScope traceScope = null;
if (tracer != null) {
traceScope = tracer.newScope(RpcClientUtil.methodToTraceString(method));
} }
RequestHeaderProto rpcRequestHeader = constructRpcRequestHeader(method); RequestHeaderProto rpcRequestHeader = constructRpcRequestHeader(method);
@ -236,9 +237,9 @@ public class ProtobufRpcEngine implements RpcEngine {
remoteId + ": " + method.getName() + remoteId + ": " + method.getName() +
" {" + e + "}"); " {" + e + "}");
} }
if (Trace.isTracing()) { if (traceScope != null) {
traceScope.getSpan().addTimelineAnnotation( traceScope.addTimelineAnnotation("Call got exception: " +
"Call got exception: " + e.toString()); e.toString());
} }
throw new ServiceException(e); throw new ServiceException(e);
} finally { } finally {

View File

@ -116,10 +116,9 @@ import org.apache.hadoop.util.ProtoUtil;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.htrace.Span; import org.apache.htrace.core.SpanId;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceInfo; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
@ -138,6 +137,7 @@ public abstract class Server {
private List<AuthMethod> enabledAuthMethods; private List<AuthMethod> enabledAuthMethods;
private RpcSaslProto negotiateResponse; private RpcSaslProto negotiateResponse;
private ExceptionsHandler exceptionsHandler = new ExceptionsHandler(); private ExceptionsHandler exceptionsHandler = new ExceptionsHandler();
private Tracer tracer;
public void addTerseExceptions(Class<?>... exceptionClass) { public void addTerseExceptions(Class<?>... exceptionClass) {
exceptionsHandler.addTerseExceptions(exceptionClass); exceptionsHandler.addTerseExceptions(exceptionClass);
@ -578,7 +578,7 @@ public abstract class Server {
private ByteBuffer rpcResponse; // the response for this call private ByteBuffer rpcResponse; // the response for this call
private final RPC.RpcKind rpcKind; private final RPC.RpcKind rpcKind;
private final byte[] clientId; private final byte[] clientId;
private final Span traceSpan; // the tracing span on the server side private final TraceScope traceScope; // the HTrace scope on the server side
public Call(int id, int retryCount, Writable param, public Call(int id, int retryCount, Writable param,
Connection connection) { Connection connection) {
@ -592,7 +592,7 @@ public abstract class Server {
} }
public Call(int id, int retryCount, Writable param, Connection connection, public Call(int id, int retryCount, Writable param, Connection connection,
RPC.RpcKind kind, byte[] clientId, Span span) { RPC.RpcKind kind, byte[] clientId, TraceScope traceScope) {
this.callId = id; this.callId = id;
this.retryCount = retryCount; this.retryCount = retryCount;
this.rpcRequest = param; this.rpcRequest = param;
@ -601,7 +601,7 @@ public abstract class Server {
this.rpcResponse = null; this.rpcResponse = null;
this.rpcKind = kind; this.rpcKind = kind;
this.clientId = clientId; this.clientId = clientId;
this.traceSpan = span; this.traceScope = traceScope;
} }
@Override @Override
@ -1929,19 +1929,24 @@ public abstract class Server {
RpcErrorCodeProto.FATAL_DESERIALIZING_REQUEST, err); RpcErrorCodeProto.FATAL_DESERIALIZING_REQUEST, err);
} }
Span traceSpan = null; TraceScope traceScope = null;
if (header.hasTraceInfo()) { if (header.hasTraceInfo()) {
// If the incoming RPC included tracing info, always continue the trace if (tracer != null) {
TraceInfo parentSpan = new TraceInfo(header.getTraceInfo().getTraceId(), // If the incoming RPC included tracing info, always continue the
header.getTraceInfo().getParentId()); // trace
traceSpan = Trace.startSpan( SpanId parentSpanId = new SpanId(
RpcClientUtil.toTraceName(rpcRequest.toString()), header.getTraceInfo().getTraceId(),
parentSpan).detach(); header.getTraceInfo().getParentId());
traceScope = tracer.newScope(
RpcClientUtil.toTraceName(rpcRequest.toString()),
parentSpanId);
traceScope.detach();
}
} }
Call call = new Call(header.getCallId(), header.getRetryCount(), Call call = new Call(header.getCallId(), header.getRetryCount(),
rpcRequest, this, ProtoUtil.convert(header.getRpcKind()), rpcRequest, this, ProtoUtil.convert(header.getRpcKind()),
header.getClientId().toByteArray(), traceSpan); header.getClientId().toByteArray(), traceScope);
if (callQueue.isClientBackoffEnabled()) { if (callQueue.isClientBackoffEnabled()) {
// if RPC queue is full, we will ask the RPC client to back off by // if RPC queue is full, we will ask the RPC client to back off by
@ -2123,8 +2128,9 @@ public abstract class Server {
Writable value = null; Writable value = null;
CurCall.set(call); CurCall.set(call);
if (call.traceSpan != null) { if (call.traceScope != null) {
traceScope = Trace.continueSpan(call.traceSpan); call.traceScope.reattach();
traceScope = call.traceScope;
traceScope.getSpan().addTimelineAnnotation("called"); traceScope.getSpan().addTimelineAnnotation("called");
} }
@ -2201,21 +2207,18 @@ public abstract class Server {
} catch (InterruptedException e) { } catch (InterruptedException e) {
if (running) { // unexpected -- log it if (running) { // unexpected -- log it
LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e); LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
if (Trace.isTracing()) { if (traceScope != null) {
traceScope.getSpan().addTimelineAnnotation("unexpectedly interrupted: " + traceScope.getSpan().addTimelineAnnotation("unexpectedly interrupted: " +
StringUtils.stringifyException(e)); StringUtils.stringifyException(e));
} }
} }
} catch (Exception e) { } catch (Exception e) {
LOG.info(Thread.currentThread().getName() + " caught an exception", e); LOG.info(Thread.currentThread().getName() + " caught an exception", e);
if (Trace.isTracing()) { if (traceScope != null) {
traceScope.getSpan().addTimelineAnnotation("Exception: " + traceScope.getSpan().addTimelineAnnotation("Exception: " +
StringUtils.stringifyException(e)); StringUtils.stringifyException(e));
} }
} finally { } finally {
if (traceScope != null) {
traceScope.close();
}
IOUtils.cleanup(LOG, traceScope); IOUtils.cleanup(LOG, traceScope);
} }
} }
@ -2529,6 +2532,10 @@ public abstract class Server {
/** Sets the socket buffer size used for responding to RPCs */ /** Sets the socket buffer size used for responding to RPCs */
public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; } public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
public void setTracer(Tracer t) {
this.tracer = t;
}
/** Starts the service. Must be called before any calls will be handled. */ /** Starts the service. Must be called before any calls will be handled. */
public synchronized void start() { public synchronized void start() {
responder.start(); responder.start();

View File

@ -42,8 +42,8 @@ import org.apache.hadoop.util.Time;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.*; import org.apache.hadoop.conf.*;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceScope; import org.apache.htrace.core.Tracer;
/** An RpcEngine implementation for Writable data. */ /** An RpcEngine implementation for Writable data. */
@InterfaceStability.Evolving @InterfaceStability.Evolving
@ -233,9 +233,14 @@ public class WritableRpcEngine implements RpcEngine {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
startTime = Time.now(); startTime = Time.now();
} }
// if Tracing is on then start a new span for this rpc.
// guard it in the if statement to make sure there isn't
// any extra string manipulation.
Tracer tracer = Tracer.curThreadTracer();
TraceScope traceScope = null; TraceScope traceScope = null;
if (Trace.isTracing()) { if (tracer != null) {
traceScope = Trace.startSpan(RpcClientUtil.methodToTraceString(method)); traceScope = tracer.newScope(RpcClientUtil.methodToTraceString(method));
} }
ObjectWritable value; ObjectWritable value;
try { try {

View File

@ -1,208 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.tracing;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
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.tracing.SpanReceiverInfo.ConfigurationPair;
import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.htrace.SpanReceiver;
import org.apache.htrace.SpanReceiverBuilder;
import org.apache.htrace.Trace;
import org.apache.htrace.impl.LocalFileSpanReceiver;
/**
* This class provides functions for reading the names of SpanReceivers from
* the Hadoop configuration, adding those SpanReceivers to the Tracer,
* and closing those SpanReceivers when appropriate.
* This class does nothing If no SpanReceiver is configured.
*/
@InterfaceAudience.Private
public class SpanReceiverHost implements TraceAdminProtocol {
public static final String SPAN_RECEIVERS_CONF_SUFFIX =
"spanreceiver.classes";
private static final Log LOG = LogFactory.getLog(SpanReceiverHost.class);
private static final HashMap<String, SpanReceiverHost> hosts =
new HashMap<String, SpanReceiverHost>(1);
private final TreeMap<Long, SpanReceiver> receivers =
new TreeMap<Long, SpanReceiver>();
private final String confPrefix;
private Configuration config;
private boolean closed = false;
private long highestId = 1;
private final static String LOCAL_FILE_SPAN_RECEIVER_PATH_SUFFIX =
"local-file-span-receiver.path";
public static SpanReceiverHost get(Configuration conf, String confPrefix) {
synchronized (SpanReceiverHost.class) {
SpanReceiverHost host = hosts.get(confPrefix);
if (host != null) {
return host;
}
final SpanReceiverHost newHost = new SpanReceiverHost(confPrefix);
newHost.loadSpanReceivers(conf);
ShutdownHookManager.get().addShutdownHook(new Runnable() {
public void run() {
newHost.closeReceivers();
}
}, 0);
hosts.put(confPrefix, newHost);
return newHost;
}
}
private static List<ConfigurationPair> EMPTY = Collections.emptyList();
private SpanReceiverHost(String confPrefix) {
this.confPrefix = confPrefix;
}
/**
* Reads the names of classes specified in the
* "hadoop.htrace.spanreceiver.classes" property and instantiates and registers
* them with the Tracer as SpanReceiver's.
*
* The nullary constructor is called during construction, but if the classes
* specified implement the Configurable interface, setConfiguration() will be
* called on them. This allows SpanReceivers to use values from the Hadoop
* configuration.
*/
public synchronized void loadSpanReceivers(Configuration conf) {
config = new Configuration(conf);
String receiverKey = confPrefix + SPAN_RECEIVERS_CONF_SUFFIX;
String[] receiverNames = config.getTrimmedStrings(receiverKey);
if (receiverNames == null || receiverNames.length == 0) {
if (LOG.isTraceEnabled()) {
LOG.trace("No span receiver names found in " + receiverKey + ".");
}
return;
}
// It's convenient to have each daemon log to a random trace file when
// testing.
String pathKey = confPrefix + LOCAL_FILE_SPAN_RECEIVER_PATH_SUFFIX;
if (config.get(pathKey) == null) {
String uniqueFile = LocalFileSpanReceiver.getUniqueLocalTraceFileName();
config.set(pathKey, uniqueFile);
if (LOG.isTraceEnabled()) {
LOG.trace("Set " + pathKey + " to " + uniqueFile);
}
}
for (String className : receiverNames) {
try {
SpanReceiver rcvr = loadInstance(className, EMPTY);
Trace.addReceiver(rcvr);
receivers.put(highestId++, rcvr);
LOG.info("Loaded SpanReceiver " + className + " successfully.");
} catch (IOException e) {
LOG.error("Failed to load SpanReceiver", e);
}
}
}
private synchronized SpanReceiver loadInstance(String className,
List<ConfigurationPair> extraConfig) throws IOException {
SpanReceiverBuilder builder =
new SpanReceiverBuilder(TraceUtils.
wrapHadoopConf(confPrefix, config, extraConfig));
SpanReceiver rcvr = builder.spanReceiverClass(className.trim()).build();
if (rcvr == null) {
throw new IOException("Failed to load SpanReceiver " + className);
}
return rcvr;
}
/**
* Calls close() on all SpanReceivers created by this SpanReceiverHost.
*/
public synchronized void closeReceivers() {
if (closed) return;
closed = true;
for (SpanReceiver rcvr : receivers.values()) {
try {
rcvr.close();
} catch (IOException e) {
LOG.warn("Unable to close SpanReceiver correctly: " + e.getMessage(), e);
}
}
receivers.clear();
}
public synchronized SpanReceiverInfo[] listSpanReceivers()
throws IOException {
SpanReceiverInfo[] info = new SpanReceiverInfo[receivers.size()];
int i = 0;
for(Map.Entry<Long, SpanReceiver> entry : receivers.entrySet()) {
info[i] = new SpanReceiverInfo(entry.getKey(),
entry.getValue().getClass().getName());
i++;
}
return info;
}
public synchronized long addSpanReceiver(SpanReceiverInfo info)
throws IOException {
StringBuilder configStringBuilder = new StringBuilder();
String prefix = "";
for (ConfigurationPair pair : info.configPairs) {
configStringBuilder.append(prefix).append(pair.getKey()).
append(" = ").append(pair.getValue());
prefix = ", ";
}
SpanReceiver rcvr = null;
try {
rcvr = loadInstance(info.getClassName(), info.configPairs);
} catch (IOException e) {
LOG.info("Failed to add SpanReceiver " + info.getClassName() +
" with configuration " + configStringBuilder.toString(), e);
throw e;
} catch (RuntimeException e) {
LOG.info("Failed to add SpanReceiver " + info.getClassName() +
" with configuration " + configStringBuilder.toString(), e);
throw e;
}
Trace.addReceiver(rcvr);
long newId = highestId++;
receivers.put(newId, rcvr);
LOG.info("Successfully added SpanReceiver " + info.getClassName() +
" with configuration " + configStringBuilder.toString());
return newId;
}
public synchronized void removeSpanReceiver(long spanReceiverId)
throws IOException {
SpanReceiver rcvr = receivers.remove(spanReceiverId);
if (rcvr == null) {
throw new IOException("There is no span receiver with id " + spanReceiverId);
}
Trace.removeReceiver(rcvr);
rcvr.close();
LOG.info("Successfully removed SpanReceiver " + spanReceiverId +
" with class " + rcvr.getClass().getName());
}
}

View File

@ -24,7 +24,7 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair; import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
import org.apache.htrace.HTraceConfiguration; import org.apache.htrace.core.HTraceConfiguration;
/** /**
* This class provides utility functions for tracing. * This class provides utility functions for tracing.
@ -32,6 +32,7 @@ import org.apache.htrace.HTraceConfiguration;
@InterfaceAudience.Private @InterfaceAudience.Private
public class TraceUtils { public class TraceUtils {
private static List<ConfigurationPair> EMPTY = Collections.emptyList(); private static List<ConfigurationPair> EMPTY = Collections.emptyList();
static final String DEFAULT_HADOOP_PREFIX = "hadoop.htrace.";
public static HTraceConfiguration wrapHadoopConf(final String prefix, public static HTraceConfiguration wrapHadoopConf(final String prefix,
final Configuration conf) { final Configuration conf) {
@ -47,16 +48,27 @@ public class TraceUtils {
return new HTraceConfiguration() { return new HTraceConfiguration() {
@Override @Override
public String get(String key) { public String get(String key) {
return get(key, ""); String ret = getInternal(prefix + key);
if (ret != null) {
return ret;
}
return getInternal(DEFAULT_HADOOP_PREFIX + key);
} }
@Override @Override
public String get(String key, String defaultValue) { public String get(String key, String defaultValue) {
String prefixedKey = prefix + key; String ret = get(key);
if (extraMap.containsKey(prefixedKey)) { if (ret != null) {
return extraMap.get(prefixedKey); return ret;
} }
return conf.get(prefixedKey, defaultValue); return defaultValue;
}
private String getInternal(String key) {
if (extraMap.containsKey(key)) {
return extraMap.get(key);
}
return conf.get(key);
} }
}; };
} }

View File

@ -0,0 +1,100 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.tracing;
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.tracing.SpanReceiverInfo.ConfigurationPair;
import org.apache.htrace.core.SpanReceiver;
import org.apache.htrace.core.TracerPool;
/**
* This class provides functions for managing the tracer configuration at
* runtime via an RPC protocol.
*/
@InterfaceAudience.Private
public class TracerConfigurationManager implements TraceAdminProtocol {
private static final Log LOG =
LogFactory.getLog(TracerConfigurationManager.class);
private final String confPrefix;
private final Configuration conf;
public TracerConfigurationManager(String confPrefix, Configuration conf) {
this.confPrefix = confPrefix;
this.conf = conf;
}
public synchronized SpanReceiverInfo[] listSpanReceivers()
throws IOException {
TracerPool pool = TracerPool.getGlobalTracerPool();
SpanReceiver[] receivers = pool.getReceivers();
SpanReceiverInfo[] info = new SpanReceiverInfo[receivers.length];
for (int i = 0; i < receivers.length; i++) {
SpanReceiver receiver = receivers[i];
info[i] = new SpanReceiverInfo(receiver.getId(),
receiver.getClass().getName());
}
return info;
}
public synchronized long addSpanReceiver(SpanReceiverInfo info)
throws IOException {
StringBuilder configStringBuilder = new StringBuilder();
String prefix = "";
for (ConfigurationPair pair : info.configPairs) {
configStringBuilder.append(prefix).append(pair.getKey()).
append(" = ").append(pair.getValue());
prefix = ", ";
}
SpanReceiver rcvr = null;
try {
rcvr = new SpanReceiver.Builder(TraceUtils.wrapHadoopConf(
confPrefix, conf, info.configPairs)).
className(info.getClassName().trim()).
build();
} catch (RuntimeException e) {
LOG.info("Failed to add SpanReceiver " + info.getClassName() +
" with configuration " + configStringBuilder.toString(), e);
throw e;
}
TracerPool.getGlobalTracerPool().addReceiver(rcvr);
LOG.info("Successfully added SpanReceiver " + info.getClassName() +
" with configuration " + configStringBuilder.toString());
return rcvr.getId();
}
public synchronized void removeSpanReceiver(long spanReceiverId)
throws IOException {
SpanReceiver[] receivers =
TracerPool.getGlobalTracerPool().getReceivers();
for (SpanReceiver receiver : receivers) {
if (receiver.getId() == spanReceiverId) {
TracerPool.getGlobalTracerPool().removeAndCloseReceiver(receiver);
LOG.info("Successfully removed SpanReceiver " + spanReceiverId +
" with class " + receiver.getClass().getName());
return;
}
}
throw new IOException("There is no span receiver with id " + spanReceiverId);
}
}

View File

@ -27,8 +27,8 @@ import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.UserInformation
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*; import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*;
import org.apache.hadoop.security.SaslRpcServer.AuthMethod; import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.htrace.Span; import org.apache.htrace.core.Span;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
@ -169,11 +169,12 @@ public abstract class ProtoUtil {
.setRetryCount(retryCount).setClientId(ByteString.copyFrom(uuid)); .setRetryCount(retryCount).setClientId(ByteString.copyFrom(uuid));
// Add tracing info if we are currently tracing. // Add tracing info if we are currently tracing.
if (Trace.isTracing()) { Span span = Tracer.getCurrentSpan();
Span s = Trace.currentSpan(); if (span != null) {
result.setTraceInfo(RPCTraceInfoProto.newBuilder() result.setTraceInfo(RPCTraceInfoProto.newBuilder()
.setParentId(s.getSpanId()) .setTraceId(span.getSpanId().getHigh())
.setTraceId(s.getTraceId()).build()); .setParentId(span.getSpanId().getLow())
.build());
} }
return result.build(); return result.build();

View File

@ -61,8 +61,9 @@ enum RpcKindProto {
* what span caused the new span we will create when this message is received. * what span caused the new span we will create when this message is received.
*/ */
message RPCTraceInfoProto { message RPCTraceInfoProto {
optional int64 traceId = 1; optional int64 traceId = 1; // parentIdHigh
optional int64 parentId = 2; optional int64 parentId = 2; // parentIdLow
} }
message RpcRequestHeaderProto { // the header for the RpcRequest message RpcRequestHeaderProto { // the header for the RpcRequest

View File

@ -2000,4 +2000,19 @@ for ldap providers in the same way as above does.
the limit is 0 or the -safely is not specified in -rm command. the limit is 0 or the -safely is not specified in -rm command.
</description> </description>
</property> </property>
<property>
<name>fs.client.htrace.sampler.classes</name>
<value></value>
<description>The class names of the HTrace Samplers to use for Hadoop
filesystem clients.
</description>
</property>
<property>
<name>hadoop.htrace.span.receiver.classes</name>
<value></value>
<description>The class names of the Span Receivers to use for Hadoop.
</description>
</property>
</configuration> </configuration>

View File

@ -49,37 +49,14 @@ interface bundled with HTrace or implementing it by yourself.
* HTracedRESTReceiver * HTracedRESTReceiver
* ZipkinSpanReceiver * ZipkinSpanReceiver
In order to set up SpanReceivers for HDFS servers, See core-default.xml for a description of HTrace configuration keys. In some
configure what SpanReceivers you'd like to use cases, you will also need to add the jar containing the SpanReceiver that you
by putting a comma separated list of the fully-qualified class name of classes implementing SpanReceiver are using to the classpath of Hadoop on each node. (In the example above,
in `hdfs-site.xml` property: `dfs.htrace.spanreceiver.classes`. LocalFileSpanReceiver is included in the htrace-core4 jar which is bundled
with Hadoop.)
```xml
<property>
<name>dfs.htrace.spanreceiver.classes</name>
<value>org.apache.htrace.impl.LocalFileSpanReceiver</value>
</property>
<property>
<name>dfs.htrace.local-file-span-receiver.path</name>
<value>/var/log/hadoop/htrace.out</value>
</property>
```
You can omit package name prefix if you use span receiver bundled with HTrace.
```xml
<property>
<name>dfs.htrace.spanreceiver.classes</name>
<value>LocalFileSpanReceiver</value>
</property>
```
You also need to add the jar bundling SpanReceiver to the classpath of Hadoop
on each node. (LocalFileSpanReceiver in the example above is included in the
jar of htrace-core which is bundled with Hadoop.)
``` ```
$ cp htrace-htraced/target/htrace-htraced-3.2.0-incubating.jar $HADOOP_HOME/share/hadoop/common/lib/ $ cp htrace-htraced/target/htrace-htraced-4.0.1-incubating.jar $HADOOP_HOME/share/hadoop/common/lib/
``` ```
### Dynamic update of tracing configuration ### Dynamic update of tracing configuration
@ -92,11 +69,11 @@ You need to run the command against all servers if you want to update the config
$ hadoop trace -list -host 192.168.56.2:9000 $ hadoop trace -list -host 192.168.56.2:9000
ID CLASS ID CLASS
1 org.apache.htrace.impl.LocalFileSpanReceiver 1 org.apache.htrace.core.LocalFileSpanReceiver
$ hadoop trace -list -host 192.168.56.2:50020 $ hadoop trace -list -host 192.168.56.2:50020
ID CLASS ID CLASS
1 org.apache.htrace.impl.LocalFileSpanReceiver 1 org.apache.htrace.core.LocalFileSpanReceiver
`hadoop trace -remove` removes span receiver from server. `hadoop trace -remove` removes span receiver from server.
`-remove` options takes id of span receiver as argument. `-remove` options takes id of span receiver as argument.
@ -113,7 +90,7 @@ You can specify the configuration associated with span receiver by `-Ckey=value`
$ hadoop trace -list -host 192.168.56.2:9000 $ hadoop trace -list -host 192.168.56.2:9000
ID CLASS ID CLASS
2 org.apache.htrace.impl.LocalFileSpanReceiver 2 org.apache.htrace.core.LocalFileSpanReceiver
### Starting tracing spans by HTrace API ### Starting tracing spans by HTrace API
@ -121,26 +98,21 @@ In order to trace, you will need to wrap the traced logic with **tracing span**
When there is running tracing spans, When there is running tracing spans,
the tracing information is propagated to servers along with RPC requests. the tracing information is propagated to servers along with RPC requests.
In addition, you need to initialize `SpanReceiverHost` once per process.
```java ```java
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.tracing.SpanReceiverHost; import org.apache.htrace.core.Tracer;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
... ...
SpanReceiverHost.getInstance(new HdfsConfiguration());
... ...
TraceScope ts = Trace.startSpan("Gets", Sampler.ALWAYS); TraceScope ts = tracer.newScope("Gets");
try { try {
... // traced logic ... // traced logic
} finally { } finally {
if (ts != null) ts.close(); ts.close();
} }
``` ```
@ -154,11 +126,10 @@ which start tracing span before invoking HDFS shell command.
import org.apache.hadoop.fs.FsShell; import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.tracing.SpanReceiverHost; import org.apache.hadoop.tracing.TraceUtils;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.apache.htrace.Sampler; import org.apache.htrace.core.Trace;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceScope;
public class TracingFsShell { public class TracingFsShell {
public static void main(String argv[]) throws Exception { public static void main(String argv[]) throws Exception {
@ -166,13 +137,19 @@ which start tracing span before invoking HDFS shell command.
FsShell shell = new FsShell(); FsShell shell = new FsShell();
conf.setQuietMode(false); conf.setQuietMode(false);
shell.setConf(conf); shell.setConf(conf);
SpanReceiverHost.get(conf, DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX); Tracer tracer = new Tracer.Builder().
name("TracingFsShell).
conf(TraceUtils.wrapHadoopConf("tracing.fs.shell.htrace.", conf)).
build();
int res = 0; int res = 0;
try (TraceScope ts = Trace.startSpan("FsShell", Sampler.ALWAYS)) { TraceScope scope = tracer.newScope("FsShell");
try {
res = ToolRunner.run(shell, argv); res = ToolRunner.run(shell, argv);
} finally { } finally {
scope.close();
shell.close(); shell.close();
} }
tracer.close();
System.exit(res); System.exit(res);
} }
} }
@ -189,16 +166,15 @@ The DFSClient can enable tracing internally. This allows you to use HTrace with
your client without modifying the client source code. your client without modifying the client source code.
Configure the span receivers and samplers in `hdfs-site.xml` Configure the span receivers and samplers in `hdfs-site.xml`
by properties `dfs.client.htrace.sampler` and `dfs.client.htrace.sampler`. by properties `fs.client.htrace.sampler.classes` and
The value of `dfs.client.htrace.sampler` can be NeverSampler, AlwaysSampler or ProbabilitySampler. `fs.client.htrace.spanreceiver.classes`. The value of
`fs.client.htrace.sampler.classes` can be NeverSampler, AlwaysSampler or
ProbabilitySampler.
* NeverSampler: HTrace is OFF for all requests to namenodes and datanodes; * NeverSampler: HTrace is OFF for all requests to namenodes and datanodes;
* AlwaysSampler: HTrace is ON for all requests to namenodes and datanodes; * AlwaysSampler: HTrace is ON for all requests to namenodes and datanodes;
* ProbabilitySampler: HTrace is ON for some percentage% of requests to namenodes and datanodes * ProbabilitySampler: HTrace is ON for some percentage% of requests to namenodes and datanodes
You do not need to enable this if your client program has been modified
to use HTrace.
```xml ```xml
<property> <property>
<name>dfs.client.htrace.spanreceiver.classes</name> <name>dfs.client.htrace.spanreceiver.classes</name>

View File

@ -24,10 +24,9 @@ import junit.framework.AssertionFailedError;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.tracing.SetSpanReceiver; import org.apache.hadoop.tracing.SetSpanReceiver;
import org.apache.hadoop.tracing.SpanReceiverHost;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.apache.htrace.SamplerBuilder; import org.apache.htrace.core.AlwaysSampler;
import org.apache.htrace.impl.AlwaysSampler; import org.apache.htrace.core.Tracer;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -53,10 +52,10 @@ public class TestFsShell {
@Test @Test
public void testTracing() throws Throwable { public void testTracing() throws Throwable {
Configuration conf = new Configuration(); Configuration conf = new Configuration();
String prefix = FsShell.SEHLL_HTRACE_PREFIX; String prefix = "fs.shell.htrace.";
conf.set(prefix + SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX, conf.set(prefix + Tracer.SPAN_RECEIVER_CLASSES_KEY,
SetSpanReceiver.class.getName()); SetSpanReceiver.class.getName());
conf.set(prefix + SamplerBuilder.SAMPLER_CONF_KEY, conf.set(prefix + Tracer.SAMPLER_CLASSES_KEY,
AlwaysSampler.class.getName()); AlwaysSampler.class.getName());
conf.setQuietMode(false); conf.setQuietMode(false);
FsShell shell = new FsShell(conf); FsShell shell = new FsShell(conf);

View File

@ -19,9 +19,10 @@ package org.apache.hadoop.tracing;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.htrace.Span; import org.apache.htrace.core.Span;
import org.apache.htrace.SpanReceiver; import org.apache.htrace.core.SpanId;
import org.apache.htrace.HTraceConfiguration; import org.apache.htrace.core.SpanReceiver;
import org.apache.htrace.core.HTraceConfiguration;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedList; import java.util.LinkedList;
@ -39,7 +40,7 @@ import org.junit.Assert;
* push all the metrics to a static place, and would make testing * push all the metrics to a static place, and would make testing
* SpanReceiverHost harder. * SpanReceiverHost harder.
*/ */
public class SetSpanReceiver implements SpanReceiver { public class SetSpanReceiver extends SpanReceiver {
public SetSpanReceiver(HTraceConfiguration conf) { public SetSpanReceiver(HTraceConfiguration conf) {
} }
@ -68,8 +69,8 @@ public class SetSpanReceiver implements SpanReceiver {
} }
public static class SetHolder { public static class SetHolder {
public static ConcurrentHashMap<Long, Span> spans = public static ConcurrentHashMap<SpanId, Span> spans =
new ConcurrentHashMap<Long, Span>(); new ConcurrentHashMap<SpanId, Span>();
public static Map<String, List<Span>> getMap() { public static Map<String, List<Span>> getMap() {
Map<String, List<Span>> map = new HashMap<String, List<Span>>(); Map<String, List<Span>> map = new HashMap<String, List<Span>>();

View File

@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals;
import java.util.LinkedList; import java.util.LinkedList;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair; import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
import org.apache.htrace.HTraceConfiguration; import org.apache.htrace.core.HTraceConfiguration;
import org.junit.Test; import org.junit.Test;
public class TestTraceUtils { public class TestTraceUtils {

View File

@ -66,6 +66,7 @@ import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -178,6 +179,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
*/ */
private Configuration configuration; private Configuration configuration;
/**
* The HTrace tracer to use.
*/
private Tracer tracer;
/** /**
* Information about the domain socket path we should use to connect to the * Information about the domain socket path we should use to connect to the
* local peer-- or null if we haven't examined the local domain socket. * local peer-- or null if we haven't examined the local domain socket.
@ -282,6 +288,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return this; return this;
} }
public BlockReaderFactory setTracer(Tracer tracer) {
this.tracer = tracer;
return this;
}
@VisibleForTesting @VisibleForTesting
public static void setFailureInjectorForTesting(FailureInjector injector) { public static void setFailureInjectorForTesting(FailureInjector injector) {
failureInjector = injector; failureInjector = injector;
@ -435,7 +446,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
try { try {
return BlockReaderLocalLegacy.newBlockReader(conf, return BlockReaderLocalLegacy.newBlockReader(conf,
userGroupInformation, configuration, fileName, block, token, userGroupInformation, configuration, fileName, block, token,
datanode, startOffset, length, storageType); datanode, startOffset, length, storageType, tracer);
} catch (RemoteException remoteException) { } catch (RemoteException remoteException) {
ioe = remoteException.unwrapRemoteException( ioe = remoteException.unwrapRemoteException(
InvalidToken.class, AccessControlException.class); InvalidToken.class, AccessControlException.class);
@ -496,6 +507,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
setVerifyChecksum(verifyChecksum). setVerifyChecksum(verifyChecksum).
setCachingStrategy(cachingStrategy). setCachingStrategy(cachingStrategy).
setStorageType(storageType). setStorageType(storageType).
setTracer(tracer).
build(); build();
} }
@ -865,12 +877,12 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return RemoteBlockReader.newBlockReader(fileName, return RemoteBlockReader.newBlockReader(fileName,
block, token, startOffset, length, conf.getIoBufferSize(), block, token, startOffset, length, conf.getIoBufferSize(),
verifyChecksum, clientName, peer, datanode, verifyChecksum, clientName, peer, datanode,
clientContext.getPeerCache(), cachingStrategy); clientContext.getPeerCache(), cachingStrategy, tracer);
} else { } else {
return RemoteBlockReader2.newBlockReader( return RemoteBlockReader2.newBlockReader(
fileName, block, token, startOffset, length, fileName, block, token, startOffset, length,
verifyChecksum, clientName, peer, datanode, verifyChecksum, clientName, peer, datanode,
clientContext.getPeerCache(), cachingStrategy); clientContext.getPeerCache(), cachingStrategy, tracer);
} }
} }

View File

@ -34,9 +34,8 @@ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DirectBufferPool; import org.apache.hadoop.util.DirectBufferPool;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -75,6 +74,7 @@ class BlockReaderLocal implements BlockReader {
private long dataPos; private long dataPos;
private ExtendedBlock block; private ExtendedBlock block;
private StorageType storageType; private StorageType storageType;
private Tracer tracer;
public Builder(ShortCircuitConf conf) { public Builder(ShortCircuitConf conf) {
this.maxReadahead = Integer.MAX_VALUE; this.maxReadahead = Integer.MAX_VALUE;
@ -120,6 +120,11 @@ class BlockReaderLocal implements BlockReader {
return this; return this;
} }
public Builder setTracer(Tracer tracer) {
this.tracer = tracer;
return this;
}
public BlockReaderLocal build() { public BlockReaderLocal build() {
Preconditions.checkNotNull(replica); Preconditions.checkNotNull(replica);
return new BlockReaderLocal(this); return new BlockReaderLocal(this);
@ -228,6 +233,11 @@ class BlockReaderLocal implements BlockReader {
*/ */
private StorageType storageType; private StorageType storageType;
/**
* The Tracer to use.
*/
private final Tracer tracer;
private BlockReaderLocal(Builder builder) { private BlockReaderLocal(Builder builder) {
this.replica = builder.replica; this.replica = builder.replica;
this.dataIn = replica.getDataStream().getChannel(); this.dataIn = replica.getDataStream().getChannel();
@ -257,6 +267,7 @@ class BlockReaderLocal implements BlockReader {
} }
this.maxReadaheadLength = maxReadaheadChunks * bytesPerChecksum; this.maxReadaheadLength = maxReadaheadChunks * bytesPerChecksum;
this.storageType = builder.storageType; this.storageType = builder.storageType;
this.tracer = builder.tracer;
} }
private synchronized void createDataBufIfNeeded() { private synchronized void createDataBufIfNeeded() {
@ -324,8 +335,8 @@ class BlockReaderLocal implements BlockReader {
*/ */
private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum) private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
throws IOException { throws IOException {
TraceScope scope = Trace.startSpan("BlockReaderLocal#fillBuffer(" + TraceScope scope = tracer.newScope(
block.getBlockId() + ")", Sampler.NEVER); "BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")");
try { try {
int total = 0; int total = 0;
long startDataPos = dataPos; long startDataPos = dataPos;

View File

@ -50,10 +50,8 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DirectBufferPool; import org.apache.hadoop.util.DirectBufferPool;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -182,7 +180,8 @@ class BlockReaderLocalLegacy implements BlockReader {
private long startOffset; private long startOffset;
private final String filename; private final String filename;
private long blockId; private long blockId;
private final Tracer tracer;
/** /**
* The only way this object can be instantiated. * The only way this object can be instantiated.
*/ */
@ -190,8 +189,8 @@ class BlockReaderLocalLegacy implements BlockReader {
UserGroupInformation userGroupInformation, UserGroupInformation userGroupInformation,
Configuration configuration, String file, ExtendedBlock blk, Configuration configuration, String file, ExtendedBlock blk,
Token<BlockTokenIdentifier> token, DatanodeInfo node, Token<BlockTokenIdentifier> token, DatanodeInfo node,
long startOffset, long length, StorageType storageType) long startOffset, long length, StorageType storageType,
throws IOException { Tracer tracer) throws IOException {
final ShortCircuitConf scConf = conf.getShortCircuitConf(); final ShortCircuitConf scConf = conf.getShortCircuitConf();
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
.getIpcPort()); .getIpcPort());
@ -239,10 +238,10 @@ class BlockReaderLocalLegacy implements BlockReader {
- (startOffset % checksum.getBytesPerChecksum()); - (startOffset % checksum.getBytesPerChecksum());
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token, localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
startOffset, length, pathinfo, checksum, true, dataIn, startOffset, length, pathinfo, checksum, true, dataIn,
firstChunkOffset, checksumIn); firstChunkOffset, checksumIn, tracer);
} else { } else {
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token, localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
startOffset, length, pathinfo, dataIn); startOffset, length, pathinfo, dataIn, tracer);
} }
} catch (IOException e) { } catch (IOException e) {
// remove from cache // remove from cache
@ -321,18 +320,18 @@ class BlockReaderLocalLegacy implements BlockReader {
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile, private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset, ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn) long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn,
throws IOException { Tracer tracer) throws IOException {
this(conf, hdfsfile, block, token, startOffset, length, pathinfo, this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false, DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
dataIn, startOffset, null); dataIn, startOffset, null, tracer);
} }
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile, private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset, ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
long length, BlockLocalPathInfo pathinfo, DataChecksum checksum, long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset, boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
FileInputStream checksumIn) throws IOException { FileInputStream checksumIn, Tracer tracer) throws IOException {
this.filename = hdfsfile; this.filename = hdfsfile;
this.checksum = checksum; this.checksum = checksum;
this.verifyChecksum = verifyChecksum; this.verifyChecksum = verifyChecksum;
@ -368,6 +367,7 @@ class BlockReaderLocalLegacy implements BlockReader {
bufferPool.returnBuffer(checksumBuff); bufferPool.returnBuffer(checksumBuff);
} }
} }
this.tracer = tracer;
} }
/** /**
@ -375,8 +375,8 @@ class BlockReaderLocalLegacy implements BlockReader {
*/ */
private int fillBuffer(FileInputStream stream, ByteBuffer buf) private int fillBuffer(FileInputStream stream, ByteBuffer buf)
throws IOException { throws IOException {
TraceScope scope = Trace.startSpan("BlockReaderLocalLegacy#fillBuffer(" + TraceScope scope = tracer.
blockId + ")", Sampler.NEVER); newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")");
try { try {
int bytesRead = stream.getChannel().read(buf); int bytesRead = stream.getChannel().read(buf);
if (bytesRead < 0) { if (bytesRead < 0) {

View File

@ -46,9 +46,9 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.htrace.Span; import org.apache.htrace.core.SpanId;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceScope; import org.apache.htrace.core.Tracer;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
@ -74,7 +74,8 @@ class BlockStorageLocationUtil {
*/ */
private static List<VolumeBlockLocationCallable> createVolumeBlockLocationCallables( private static List<VolumeBlockLocationCallable> createVolumeBlockLocationCallables(
Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks, Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
int timeout, boolean connectToDnViaHostname, Span parent) { int timeout, boolean connectToDnViaHostname,
Tracer tracer, SpanId parentSpanId) {
if (datanodeBlocks.isEmpty()) { if (datanodeBlocks.isEmpty()) {
return Lists.newArrayList(); return Lists.newArrayList();
@ -114,7 +115,7 @@ class BlockStorageLocationUtil {
} }
VolumeBlockLocationCallable callable = new VolumeBlockLocationCallable( VolumeBlockLocationCallable callable = new VolumeBlockLocationCallable(
conf, datanode, poolId, blockIds, dnTokens, timeout, conf, datanode, poolId, blockIds, dnTokens, timeout,
connectToDnViaHostname, parent); connectToDnViaHostname, tracer, parentSpanId);
callables.add(callable); callables.add(callable);
} }
return callables; return callables;
@ -133,12 +134,12 @@ class BlockStorageLocationUtil {
*/ */
static Map<DatanodeInfo, HdfsBlocksMetadata> queryDatanodesForHdfsBlocksMetadata( static Map<DatanodeInfo, HdfsBlocksMetadata> queryDatanodesForHdfsBlocksMetadata(
Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks, Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
int poolsize, int timeoutMs, boolean connectToDnViaHostname) int poolsize, int timeoutMs, boolean connectToDnViaHostname,
throws InvalidBlockTokenException { Tracer tracer, SpanId parentSpanId) throws InvalidBlockTokenException {
List<VolumeBlockLocationCallable> callables = List<VolumeBlockLocationCallable> callables =
createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs, createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs,
connectToDnViaHostname, Trace.currentSpan()); connectToDnViaHostname, tracer, parentSpanId);
// Use a thread pool to execute the Callables in parallel // Use a thread pool to execute the Callables in parallel
List<Future<HdfsBlocksMetadata>> futures = List<Future<HdfsBlocksMetadata>> futures =
@ -322,12 +323,14 @@ class BlockStorageLocationUtil {
private final long[] blockIds; private final long[] blockIds;
private final List<Token<BlockTokenIdentifier>> dnTokens; private final List<Token<BlockTokenIdentifier>> dnTokens;
private final boolean connectToDnViaHostname; private final boolean connectToDnViaHostname;
private final Span parentSpan; private final Tracer tracer;
private final SpanId parentSpanId;
VolumeBlockLocationCallable(Configuration configuration, VolumeBlockLocationCallable(Configuration configuration,
DatanodeInfo datanode, String poolId, long []blockIds, DatanodeInfo datanode, String poolId, long []blockIds,
List<Token<BlockTokenIdentifier>> dnTokens, int timeout, List<Token<BlockTokenIdentifier>> dnTokens, int timeout,
boolean connectToDnViaHostname, Span parentSpan) { boolean connectToDnViaHostname,
Tracer tracer, SpanId parentSpanId) {
this.configuration = configuration; this.configuration = configuration;
this.timeout = timeout; this.timeout = timeout;
this.datanode = datanode; this.datanode = datanode;
@ -335,7 +338,8 @@ class BlockStorageLocationUtil {
this.blockIds = blockIds; this.blockIds = blockIds;
this.dnTokens = dnTokens; this.dnTokens = dnTokens;
this.connectToDnViaHostname = connectToDnViaHostname; this.connectToDnViaHostname = connectToDnViaHostname;
this.parentSpan = parentSpan; this.tracer = tracer;
this.parentSpanId = parentSpanId;
} }
public DatanodeInfo getDatanodeInfo() { public DatanodeInfo getDatanodeInfo() {
@ -348,7 +352,7 @@ class BlockStorageLocationUtil {
// Create the RPC proxy and make the RPC // Create the RPC proxy and make the RPC
ClientDatanodeProtocol cdp = null; ClientDatanodeProtocol cdp = null;
TraceScope scope = TraceScope scope =
Trace.startSpan("getHdfsBlocksMetadata", parentSpan); tracer.newScope("getHdfsBlocksMetadata", parentSpanId);
try { try {
cdp = DFSUtilClient.createClientDatanodeProtocolProxy( cdp = DFSUtilClient.createClientDatanodeProtocolProxy(
datanode, configuration, datanode, configuration,

View File

@ -77,6 +77,7 @@ import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.FsStatus; import org.apache.hadoop.fs.FsStatus;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.fs.HdfsBlockLocation; import org.apache.hadoop.fs.HdfsBlockLocation;
import org.apache.hadoop.fs.InvalidPathException; import org.apache.hadoop.fs.InvalidPathException;
import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum; import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
@ -172,24 +173,19 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer; import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.tracing.SpanReceiverHost;
import org.apache.hadoop.tracing.TraceUtils;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DataChecksum.Type; import org.apache.hadoop.util.DataChecksum.Type;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.SamplerBuilder;
import org.apache.htrace.Span;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.net.InetAddresses; import com.google.common.net.InetAddresses;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -212,6 +208,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
private final Configuration conf; private final Configuration conf;
private final Tracer tracer;
private final DfsClientConf dfsClientConf; private final DfsClientConf dfsClientConf;
final ClientProtocol namenode; final ClientProtocol namenode;
/* The service used for delegation tokens */ /* The service used for delegation tokens */
@ -238,7 +235,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
private static final DFSHedgedReadMetrics HEDGED_READ_METRIC = private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
new DFSHedgedReadMetrics(); new DFSHedgedReadMetrics();
private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL; private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
private final Sampler<?> traceSampler;
private final int smallBufferSize; private final int smallBufferSize;
public DfsClientConf getConf() { public DfsClientConf getConf() {
@ -302,11 +298,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode, public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
Configuration conf, FileSystem.Statistics stats) Configuration conf, FileSystem.Statistics stats)
throws IOException { throws IOException {
SpanReceiverHost.get(conf, HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX);
traceSampler = new SamplerBuilder(TraceUtils.
wrapHadoopConf(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX, conf))
.build();
// Copy only the required DFSClient configuration // Copy only the required DFSClient configuration
this.tracer = FsTracer.get(conf);
this.dfsClientConf = new DfsClientConf(conf); this.dfsClientConf = new DfsClientConf(conf);
this.conf = conf; this.conf = conf;
this.stats = stats; this.stats = stats;
@ -628,7 +621,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
* @see ClientProtocol#getPreferredBlockSize(String) * @see ClientProtocol#getPreferredBlockSize(String)
*/ */
public long getBlockSize(String f) throws IOException { public long getBlockSize(String f) throws IOException {
TraceScope scope = getPathTraceScope("getBlockSize", f); TraceScope scope = newPathTraceScope("getBlockSize", f);
try { try {
return namenode.getPreferredBlockSize(f); return namenode.getPreferredBlockSize(f);
} catch (IOException ie) { } catch (IOException ie) {
@ -670,7 +663,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
throws IOException { throws IOException {
assert dtService != null; assert dtService != null;
TraceScope scope = Trace.startSpan("getDelegationToken", traceSampler); TraceScope scope = tracer.newScope("getDelegationToken");
try { try {
Token<DelegationTokenIdentifier> token = Token<DelegationTokenIdentifier> token =
namenode.getDelegationToken(renewer); namenode.getDelegationToken(renewer);
@ -826,7 +819,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
@VisibleForTesting @VisibleForTesting
public LocatedBlocks getLocatedBlocks(String src, long start, long length) public LocatedBlocks getLocatedBlocks(String src, long start, long length)
throws IOException { throws IOException {
TraceScope scope = getPathTraceScope("getBlockLocations", src); TraceScope scope = newPathTraceScope("getBlockLocations", src);
try { try {
return callGetBlockLocations(namenode, src, start, length); return callGetBlockLocations(namenode, src, start, length);
} finally { } finally {
@ -858,7 +851,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
boolean recoverLease(String src) throws IOException { boolean recoverLease(String src) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("recoverLease", src); TraceScope scope = newPathTraceScope("recoverLease", src);
try { try {
return namenode.recoverLease(src, clientName); return namenode.recoverLease(src, clientName);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -884,7 +877,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public BlockLocation[] getBlockLocations(String src, long start, public BlockLocation[] getBlockLocations(String src, long start,
long length) throws IOException, UnresolvedLinkException { long length) throws IOException, UnresolvedLinkException {
TraceScope scope = getPathTraceScope("getBlockLocations", src); TraceScope scope = newPathTraceScope("getBlockLocations", src);
try { try {
LocatedBlocks blocks = getLocatedBlocks(src, start, length); LocatedBlocks blocks = getLocatedBlocks(src, start, length);
BlockLocation[] locations = DFSUtilClient.locatedBlocks2Locations(blocks); BlockLocation[] locations = DFSUtilClient.locatedBlocks2Locations(blocks);
@ -950,14 +943,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
// Make RPCs to the datanodes to get volume locations for its replicas // Make RPCs to the datanodes to get volume locations for its replicas
TraceScope scope = TraceScope scope =
Trace.startSpan("getBlockStorageLocations", traceSampler); tracer.newScope("getBlockStorageLocations");
Map<DatanodeInfo, HdfsBlocksMetadata> metadatas; Map<DatanodeInfo, HdfsBlocksMetadata> metadatas;
try { try {
metadatas = BlockStorageLocationUtil. metadatas = BlockStorageLocationUtil.
queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks, queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
getConf().getFileBlockStorageLocationsNumThreads(), getConf().getFileBlockStorageLocationsNumThreads(),
getConf().getFileBlockStorageLocationsTimeoutMs(), getConf().getFileBlockStorageLocationsTimeoutMs(),
getConf().isConnectToDnViaHostname()); getConf().isConnectToDnViaHostname(), tracer, scope.getSpanId());
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace("metadata returned: " LOG.trace("metadata returned: "
+ Joiner.on("\n").withKeyValueSeparator("=").join(metadatas)); + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
@ -983,7 +976,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
feInfo) throws IOException { feInfo) throws IOException {
TraceScope scope = Trace.startSpan("decryptEDEK", traceSampler); TraceScope scope = tracer.newScope("decryptEDEK");
try { try {
KeyProvider provider = getKeyProvider(); KeyProvider provider = getKeyProvider();
if (provider == null) { if (provider == null) {
@ -1139,7 +1132,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
throws IOException, UnresolvedLinkException { throws IOException, UnresolvedLinkException {
checkOpen(); checkOpen();
// Get block info from namenode // Get block info from namenode
TraceScope scope = getPathTraceScope("newDFSInputStream", src); TraceScope scope = newPathTraceScope("newDFSInputStream", src);
try { try {
return new DFSInputStream(this, src, verifyChecksum, null); return new DFSInputStream(this, src, verifyChecksum, null);
} finally { } finally {
@ -1384,7 +1377,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public void createSymlink(String target, String link, boolean createParent) public void createSymlink(String target, String link, boolean createParent)
throws IOException { throws IOException {
TraceScope scope = getPathTraceScope("createSymlink", target); TraceScope scope = newPathTraceScope("createSymlink", target);
try { try {
final FsPermission dirPerm = applyUMask(null); final FsPermission dirPerm = applyUMask(null);
namenode.createSymlink(target, link, dirPerm, createParent); namenode.createSymlink(target, link, dirPerm, createParent);
@ -1410,7 +1403,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public String getLinkTarget(String path) throws IOException { public String getLinkTarget(String path) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("getLinkTarget", path); TraceScope scope = newPathTraceScope("getLinkTarget", path);
try { try {
return namenode.getLinkTarget(path); return namenode.getLinkTarget(path);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -1506,7 +1499,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public boolean setReplication(String src, short replication) public boolean setReplication(String src, short replication)
throws IOException { throws IOException {
TraceScope scope = getPathTraceScope("setReplication", src); TraceScope scope = newPathTraceScope("setReplication", src);
try { try {
return namenode.setReplication(src, replication); return namenode.setReplication(src, replication);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -1529,7 +1522,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public void setStoragePolicy(String src, String policyName) public void setStoragePolicy(String src, String policyName)
throws IOException { throws IOException {
TraceScope scope = getPathTraceScope("setStoragePolicy", src); TraceScope scope = newPathTraceScope("setStoragePolicy", src);
try { try {
namenode.setStoragePolicy(src, policyName); namenode.setStoragePolicy(src, policyName);
} catch (RemoteException e) { } catch (RemoteException e) {
@ -1550,7 +1543,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public BlockStoragePolicy getStoragePolicy(String path) throws IOException { public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("getStoragePolicy", path); TraceScope scope = newPathTraceScope("getStoragePolicy", path);
try { try {
return namenode.getStoragePolicy(path); return namenode.getStoragePolicy(path);
} catch (RemoteException e) { } catch (RemoteException e) {
@ -1567,7 +1560,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
* @return All the existing storage policies * @return All the existing storage policies
*/ */
public BlockStoragePolicy[] getStoragePolicies() throws IOException { public BlockStoragePolicy[] getStoragePolicies() throws IOException {
TraceScope scope = Trace.startSpan("getStoragePolicies", traceSampler); TraceScope scope = tracer.newScope("getStoragePolicies");
try { try {
return namenode.getStoragePolicies(); return namenode.getStoragePolicies();
} finally { } finally {
@ -1583,7 +1576,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
@Deprecated @Deprecated
public boolean rename(String src, String dst) throws IOException { public boolean rename(String src, String dst) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getSrcDstTraceScope("rename", src, dst); TraceScope scope = newSrcDstTraceScope("rename", src, dst);
try { try {
return namenode.rename(src, dst); return namenode.rename(src, dst);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -1604,7 +1597,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public void concat(String trg, String [] srcs) throws IOException { public void concat(String trg, String [] srcs) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("concat", traceSampler); TraceScope scope = tracer.newScope("concat");
try { try {
namenode.concat(trg, srcs); namenode.concat(trg, srcs);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -1622,7 +1615,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void rename(String src, String dst, Options.Rename... options) public void rename(String src, String dst, Options.Rename... options)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getSrcDstTraceScope("rename2", src, dst); TraceScope scope = newSrcDstTraceScope("rename2", src, dst);
try { try {
namenode.rename2(src, dst, options); namenode.rename2(src, dst, options);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -1651,11 +1644,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
throw new HadoopIllegalArgumentException( throw new HadoopIllegalArgumentException(
"Cannot truncate to a negative file size: " + newLength + "."); "Cannot truncate to a negative file size: " + newLength + ".");
} }
TraceScope scope = newPathTraceScope("truncate", src);
try { try {
return namenode.truncate(src, newLength, clientName); return namenode.truncate(src, newLength, clientName);
} catch (RemoteException re) { } catch (RemoteException re) {
throw re.unwrapRemoteException(AccessControlException.class, throw re.unwrapRemoteException(AccessControlException.class,
UnresolvedPathException.class); UnresolvedPathException.class);
} finally {
scope.close();
} }
} }
@ -1678,7 +1674,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public boolean delete(String src, boolean recursive) throws IOException { public boolean delete(String src, boolean recursive) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("delete", src); TraceScope scope = newPathTraceScope("delete", src);
try { try {
return namenode.delete(src, recursive); return namenode.delete(src, recursive);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -1720,7 +1716,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public DirectoryListing listPaths(String src, byte[] startAfter, public DirectoryListing listPaths(String src, byte[] startAfter,
boolean needLocation) throws IOException { boolean needLocation) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("listPaths", src); TraceScope scope = newPathTraceScope("listPaths", src);
try { try {
return namenode.getListing(src, startAfter, needLocation); return namenode.getListing(src, startAfter, needLocation);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -1742,7 +1738,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public HdfsFileStatus getFileInfo(String src) throws IOException { public HdfsFileStatus getFileInfo(String src) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("getFileInfo", src); TraceScope scope = newPathTraceScope("getFileInfo", src);
try { try {
return namenode.getFileInfo(src); return namenode.getFileInfo(src);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -1760,7 +1756,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public boolean isFileClosed(String src) throws IOException{ public boolean isFileClosed(String src) throws IOException{
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("isFileClosed", src); TraceScope scope = newPathTraceScope("isFileClosed", src);
try { try {
return namenode.isFileClosed(src); return namenode.isFileClosed(src);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -1782,7 +1778,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public HdfsFileStatus getFileLinkInfo(String src) throws IOException { public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("getFileLinkInfo", src); TraceScope scope = newPathTraceScope("getFileLinkInfo", src);
try { try {
return namenode.getFileLinkInfo(src); return namenode.getFileLinkInfo(src);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2085,7 +2081,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void setPermission(String src, FsPermission permission) public void setPermission(String src, FsPermission permission)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("setPermission", src); TraceScope scope = newPathTraceScope("setPermission", src);
try { try {
namenode.setPermission(src, permission); namenode.setPermission(src, permission);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2110,7 +2106,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void setOwner(String src, String username, String groupname) public void setOwner(String src, String username, String groupname)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("setOwner", src); TraceScope scope = newPathTraceScope("setOwner", src);
try { try {
namenode.setOwner(src, username, groupname); namenode.setOwner(src, username, groupname);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2126,7 +2122,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
private long[] callGetStats() throws IOException { private long[] callGetStats() throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("getStats", traceSampler); TraceScope scope = tracer.newScope("getStats");
try { try {
return namenode.getStats(); return namenode.getStats();
} finally { } finally {
@ -2185,7 +2181,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
String cookie) String cookie)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("listCorruptFileBlocks", path); TraceScope scope = newPathTraceScope("listCorruptFileBlocks", path);
try { try {
return namenode.listCorruptFileBlocks(path, cookie); return namenode.listCorruptFileBlocks(path, cookie);
} finally { } finally {
@ -2196,7 +2192,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public DatanodeInfo[] datanodeReport(DatanodeReportType type) public DatanodeInfo[] datanodeReport(DatanodeReportType type)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("datanodeReport", traceSampler); TraceScope scope = tracer.newScope("datanodeReport");
try { try {
return namenode.getDatanodeReport(type); return namenode.getDatanodeReport(type);
} finally { } finally {
@ -2208,7 +2204,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
DatanodeReportType type) throws IOException { DatanodeReportType type) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = TraceScope scope =
Trace.startSpan("datanodeStorageReport", traceSampler); tracer.newScope("datanodeStorageReport");
try { try {
return namenode.getDatanodeStorageReport(type); return namenode.getDatanodeStorageReport(type);
} finally { } finally {
@ -2238,7 +2234,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{ public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
TraceScope scope = TraceScope scope =
Trace.startSpan("setSafeMode", traceSampler); tracer.newScope("setSafeMode");
try { try {
return namenode.setSafeMode(action, isChecked); return namenode.setSafeMode(action, isChecked);
} finally { } finally {
@ -2257,7 +2253,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public String createSnapshot(String snapshotRoot, String snapshotName) public String createSnapshot(String snapshotRoot, String snapshotName)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("createSnapshot", traceSampler); TraceScope scope = tracer.newScope("createSnapshot");
try { try {
return namenode.createSnapshot(snapshotRoot, snapshotName); return namenode.createSnapshot(snapshotRoot, snapshotName);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2279,7 +2275,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void deleteSnapshot(String snapshotRoot, String snapshotName) public void deleteSnapshot(String snapshotRoot, String snapshotName)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("deleteSnapshot", traceSampler); TraceScope scope = tracer.newScope("deleteSnapshot");
try { try {
namenode.deleteSnapshot(snapshotRoot, snapshotName); namenode.deleteSnapshot(snapshotRoot, snapshotName);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2300,7 +2296,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void renameSnapshot(String snapshotDir, String snapshotOldName, public void renameSnapshot(String snapshotDir, String snapshotOldName,
String snapshotNewName) throws IOException { String snapshotNewName) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("renameSnapshot", traceSampler); TraceScope scope = tracer.newScope("renameSnapshot");
try { try {
namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName); namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2319,8 +2315,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public SnapshottableDirectoryStatus[] getSnapshottableDirListing() public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("getSnapshottableDirListing", TraceScope scope = tracer.newScope("getSnapshottableDirListing");
traceSampler);
try { try {
return namenode.getSnapshottableDirListing(); return namenode.getSnapshottableDirListing();
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2337,7 +2332,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public void allowSnapshot(String snapshotRoot) throws IOException { public void allowSnapshot(String snapshotRoot) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("allowSnapshot", traceSampler); TraceScope scope = tracer.newScope("allowSnapshot");
try { try {
namenode.allowSnapshot(snapshotRoot); namenode.allowSnapshot(snapshotRoot);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2354,7 +2349,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public void disallowSnapshot(String snapshotRoot) throws IOException { public void disallowSnapshot(String snapshotRoot) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("disallowSnapshot", traceSampler); TraceScope scope = tracer.newScope("disallowSnapshot");
try { try {
namenode.disallowSnapshot(snapshotRoot); namenode.disallowSnapshot(snapshotRoot);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2372,7 +2367,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir, public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir,
String fromSnapshot, String toSnapshot) throws IOException { String fromSnapshot, String toSnapshot) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("getSnapshotDiffReport", traceSampler); TraceScope scope = tracer.newScope("getSnapshotDiffReport");
try { try {
return namenode.getSnapshotDiffReport(snapshotDir, return namenode.getSnapshotDiffReport(snapshotDir,
fromSnapshot, toSnapshot); fromSnapshot, toSnapshot);
@ -2386,7 +2381,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public long addCacheDirective( public long addCacheDirective(
CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException { CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("addCacheDirective", traceSampler); TraceScope scope = tracer.newScope("addCacheDirective");
try { try {
return namenode.addCacheDirective(info, flags); return namenode.addCacheDirective(info, flags);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2399,7 +2394,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void modifyCacheDirective( public void modifyCacheDirective(
CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException { CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("modifyCacheDirective", traceSampler); TraceScope scope = tracer.newScope("modifyCacheDirective");
try { try {
namenode.modifyCacheDirective(info, flags); namenode.modifyCacheDirective(info, flags);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2412,7 +2407,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void removeCacheDirective(long id) public void removeCacheDirective(long id)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("removeCacheDirective", traceSampler); TraceScope scope = tracer.newScope("removeCacheDirective");
try { try {
namenode.removeCacheDirective(id); namenode.removeCacheDirective(id);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2424,12 +2419,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public RemoteIterator<CacheDirectiveEntry> listCacheDirectives( public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
CacheDirectiveInfo filter) throws IOException { CacheDirectiveInfo filter) throws IOException {
return new CacheDirectiveIterator(namenode, filter, traceSampler); return new CacheDirectiveIterator(namenode, filter, tracer);
} }
public void addCachePool(CachePoolInfo info) throws IOException { public void addCachePool(CachePoolInfo info) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("addCachePool", traceSampler); TraceScope scope = tracer.newScope("addCachePool");
try { try {
namenode.addCachePool(info); namenode.addCachePool(info);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2441,7 +2436,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void modifyCachePool(CachePoolInfo info) throws IOException { public void modifyCachePool(CachePoolInfo info) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("modifyCachePool", traceSampler); TraceScope scope = tracer.newScope("modifyCachePool");
try { try {
namenode.modifyCachePool(info); namenode.modifyCachePool(info);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2453,7 +2448,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void removeCachePool(String poolName) throws IOException { public void removeCachePool(String poolName) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("removeCachePool", traceSampler); TraceScope scope = tracer.newScope("removeCachePool");
try { try {
namenode.removeCachePool(poolName); namenode.removeCachePool(poolName);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2464,7 +2459,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
} }
public RemoteIterator<CachePoolEntry> listCachePools() throws IOException { public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
return new CachePoolIterator(namenode, traceSampler); return new CachePoolIterator(namenode, tracer);
} }
/** /**
@ -2473,7 +2468,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
* @see ClientProtocol#saveNamespace() * @see ClientProtocol#saveNamespace()
*/ */
void saveNamespace() throws AccessControlException, IOException { void saveNamespace() throws AccessControlException, IOException {
TraceScope scope = Trace.startSpan("saveNamespace", traceSampler); TraceScope scope = tracer.newScope("saveNamespace");
try { try {
namenode.saveNamespace(); namenode.saveNamespace();
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2490,7 +2485,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
* @see ClientProtocol#rollEdits() * @see ClientProtocol#rollEdits()
*/ */
long rollEdits() throws AccessControlException, IOException { long rollEdits() throws AccessControlException, IOException {
TraceScope scope = Trace.startSpan("rollEdits", traceSampler); TraceScope scope = tracer.newScope("rollEdits");
try { try {
return namenode.rollEdits(); return namenode.rollEdits();
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2512,7 +2507,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
boolean restoreFailedStorage(String arg) boolean restoreFailedStorage(String arg)
throws AccessControlException, IOException{ throws AccessControlException, IOException{
TraceScope scope = Trace.startSpan("restoreFailedStorage", traceSampler); TraceScope scope = tracer.newScope("restoreFailedStorage");
try { try {
return namenode.restoreFailedStorage(arg); return namenode.restoreFailedStorage(arg);
} finally { } finally {
@ -2528,7 +2523,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
* @see ClientProtocol#refreshNodes() * @see ClientProtocol#refreshNodes()
*/ */
public void refreshNodes() throws IOException { public void refreshNodes() throws IOException {
TraceScope scope = Trace.startSpan("refreshNodes", traceSampler); TraceScope scope = tracer.newScope("refreshNodes");
try { try {
namenode.refreshNodes(); namenode.refreshNodes();
} finally { } finally {
@ -2542,7 +2537,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
* @see ClientProtocol#metaSave(String) * @see ClientProtocol#metaSave(String)
*/ */
public void metaSave(String pathname) throws IOException { public void metaSave(String pathname) throws IOException {
TraceScope scope = Trace.startSpan("metaSave", traceSampler); TraceScope scope = tracer.newScope("metaSave");
try { try {
namenode.metaSave(pathname); namenode.metaSave(pathname);
} finally { } finally {
@ -2559,7 +2554,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
* @see ClientProtocol#setBalancerBandwidth(long) * @see ClientProtocol#setBalancerBandwidth(long)
*/ */
public void setBalancerBandwidth(long bandwidth) throws IOException { public void setBalancerBandwidth(long bandwidth) throws IOException {
TraceScope scope = Trace.startSpan("setBalancerBandwidth", traceSampler); TraceScope scope = tracer.newScope("setBalancerBandwidth");
try { try {
namenode.setBalancerBandwidth(bandwidth); namenode.setBalancerBandwidth(bandwidth);
} finally { } finally {
@ -2571,7 +2566,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
* @see ClientProtocol#finalizeUpgrade() * @see ClientProtocol#finalizeUpgrade()
*/ */
public void finalizeUpgrade() throws IOException { public void finalizeUpgrade() throws IOException {
TraceScope scope = Trace.startSpan("finalizeUpgrade", traceSampler); TraceScope scope = tracer.newScope("finalizeUpgrade");
try { try {
namenode.finalizeUpgrade(); namenode.finalizeUpgrade();
} finally { } finally {
@ -2580,7 +2575,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
} }
RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException { RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
TraceScope scope = Trace.startSpan("rollingUpgrade", traceSampler); TraceScope scope = tracer.newScope("rollingUpgrade");
try { try {
return namenode.rollingUpgrade(action); return namenode.rollingUpgrade(action);
} finally { } finally {
@ -2638,7 +2633,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
LOG.debug(src + ": masked=" + absPermission); LOG.debug(src + ": masked=" + absPermission);
} }
TraceScope scope = Trace.startSpan("mkdir", traceSampler); TraceScope scope = tracer.newScope("mkdir");
try { try {
return namenode.mkdirs(src, absPermission, createParent); return namenode.mkdirs(src, absPermission, createParent);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2665,7 +2660,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
* @see ClientProtocol#getContentSummary(String) * @see ClientProtocol#getContentSummary(String)
*/ */
ContentSummary getContentSummary(String src) throws IOException { ContentSummary getContentSummary(String src) throws IOException {
TraceScope scope = getPathTraceScope("getContentSummary", src); TraceScope scope = newPathTraceScope("getContentSummary", src);
try { try {
return namenode.getContentSummary(src); return namenode.getContentSummary(src);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2693,7 +2688,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
storagespaceQuota); storagespaceQuota);
} }
TraceScope scope = getPathTraceScope("setQuota", src); TraceScope scope = newPathTraceScope("setQuota", src);
try { try {
// Pass null as storage type for traditional namespace/storagespace quota. // Pass null as storage type for traditional namespace/storagespace quota.
namenode.setQuota(src, namespaceQuota, storagespaceQuota, null); namenode.setQuota(src, namespaceQuota, storagespaceQuota, null);
@ -2728,7 +2723,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
throw new IllegalArgumentException("Don't support Quota for storage type : " throw new IllegalArgumentException("Don't support Quota for storage type : "
+ type.toString()); + type.toString());
} }
TraceScope scope = getPathTraceScope("setQuotaByStorageType", src); TraceScope scope = newPathTraceScope("setQuotaByStorageType", src);
try { try {
namenode.setQuota(src, HdfsConstants.QUOTA_DONT_SET, quota, type); namenode.setQuota(src, HdfsConstants.QUOTA_DONT_SET, quota, type);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2748,7 +2743,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
*/ */
public void setTimes(String src, long mtime, long atime) throws IOException { public void setTimes(String src, long mtime, long atime) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("setTimes", src); TraceScope scope = newPathTraceScope("setTimes", src);
try { try {
namenode.setTimes(src, mtime, atime); namenode.setTimes(src, mtime, atime);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2809,7 +2804,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void modifyAclEntries(String src, List<AclEntry> aclSpec) public void modifyAclEntries(String src, List<AclEntry> aclSpec)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("modifyAclEntries", src); TraceScope scope = newPathTraceScope("modifyAclEntries", src);
try { try {
namenode.modifyAclEntries(src, aclSpec); namenode.modifyAclEntries(src, aclSpec);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2828,7 +2823,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void removeAclEntries(String src, List<AclEntry> aclSpec) public void removeAclEntries(String src, List<AclEntry> aclSpec)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("removeAclEntries", traceSampler); TraceScope scope = tracer.newScope("removeAclEntries");
try { try {
namenode.removeAclEntries(src, aclSpec); namenode.removeAclEntries(src, aclSpec);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2846,7 +2841,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void removeDefaultAcl(String src) throws IOException { public void removeDefaultAcl(String src) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("removeDefaultAcl", traceSampler); TraceScope scope = tracer.newScope("removeDefaultAcl");
try { try {
namenode.removeDefaultAcl(src); namenode.removeDefaultAcl(src);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2864,7 +2859,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void removeAcl(String src) throws IOException { public void removeAcl(String src) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("removeAcl", traceSampler); TraceScope scope = tracer.newScope("removeAcl");
try { try {
namenode.removeAcl(src); namenode.removeAcl(src);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2882,7 +2877,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void setAcl(String src, List<AclEntry> aclSpec) throws IOException { public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = Trace.startSpan("setAcl", traceSampler); TraceScope scope = tracer.newScope("setAcl");
try { try {
namenode.setAcl(src, aclSpec); namenode.setAcl(src, aclSpec);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2900,7 +2895,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public AclStatus getAclStatus(String src) throws IOException { public AclStatus getAclStatus(String src) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("getAclStatus", src); TraceScope scope = newPathTraceScope("getAclStatus", src);
try { try {
return namenode.getAclStatus(src); return namenode.getAclStatus(src);
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2916,7 +2911,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void createEncryptionZone(String src, String keyName) public void createEncryptionZone(String src, String keyName)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("createEncryptionZone", src); TraceScope scope = newPathTraceScope("createEncryptionZone", src);
try { try {
namenode.createEncryptionZone(src, keyName); namenode.createEncryptionZone(src, keyName);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2931,7 +2926,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public EncryptionZone getEZForPath(String src) public EncryptionZone getEZForPath(String src)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("getEZForPath", src); TraceScope scope = newPathTraceScope("getEZForPath", src);
try { try {
return namenode.getEZForPath(src); return namenode.getEZForPath(src);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2945,13 +2940,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public RemoteIterator<EncryptionZone> listEncryptionZones() public RemoteIterator<EncryptionZone> listEncryptionZones()
throws IOException { throws IOException {
checkOpen(); checkOpen();
return new EncryptionZoneIterator(namenode, traceSampler); return new EncryptionZoneIterator(namenode, tracer);
} }
public void setXAttr(String src, String name, byte[] value, public void setXAttr(String src, String name, byte[] value,
EnumSet<XAttrSetFlag> flag) throws IOException { EnumSet<XAttrSetFlag> flag) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("setXAttr", src); TraceScope scope = newPathTraceScope("setXAttr", src);
try { try {
namenode.setXAttr(src, XAttrHelper.buildXAttr(name, value), flag); namenode.setXAttr(src, XAttrHelper.buildXAttr(name, value), flag);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -2968,7 +2963,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public byte[] getXAttr(String src, String name) throws IOException { public byte[] getXAttr(String src, String name) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("getXAttr", src); TraceScope scope = newPathTraceScope("getXAttr", src);
try { try {
final List<XAttr> xAttrs = XAttrHelper.buildXAttrAsList(name); final List<XAttr> xAttrs = XAttrHelper.buildXAttrAsList(name);
final List<XAttr> result = namenode.getXAttrs(src, xAttrs); final List<XAttr> result = namenode.getXAttrs(src, xAttrs);
@ -2984,7 +2979,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public Map<String, byte[]> getXAttrs(String src) throws IOException { public Map<String, byte[]> getXAttrs(String src) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("getXAttrs", src); TraceScope scope = newPathTraceScope("getXAttrs", src);
try { try {
return XAttrHelper.buildXAttrMap(namenode.getXAttrs(src, null)); return XAttrHelper.buildXAttrMap(namenode.getXAttrs(src, null));
} catch(RemoteException re) { } catch(RemoteException re) {
@ -2999,7 +2994,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public Map<String, byte[]> getXAttrs(String src, List<String> names) public Map<String, byte[]> getXAttrs(String src, List<String> names)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("getXAttrs", src); TraceScope scope = newPathTraceScope("getXAttrs", src);
try { try {
return XAttrHelper.buildXAttrMap(namenode.getXAttrs( return XAttrHelper.buildXAttrMap(namenode.getXAttrs(
src, XAttrHelper.buildXAttrs(names))); src, XAttrHelper.buildXAttrs(names)));
@ -3015,7 +3010,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public List<String> listXAttrs(String src) public List<String> listXAttrs(String src)
throws IOException { throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("listXAttrs", src); TraceScope scope = newPathTraceScope("listXAttrs", src);
try { try {
final Map<String, byte[]> xattrs = final Map<String, byte[]> xattrs =
XAttrHelper.buildXAttrMap(namenode.listXAttrs(src)); XAttrHelper.buildXAttrMap(namenode.listXAttrs(src));
@ -3031,7 +3026,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void removeXAttr(String src, String name) throws IOException { public void removeXAttr(String src, String name) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("removeXAttr", src); TraceScope scope = newPathTraceScope("removeXAttr", src);
try { try {
namenode.removeXAttr(src, XAttrHelper.buildXAttr(name)); namenode.removeXAttr(src, XAttrHelper.buildXAttr(name));
} catch(RemoteException re) { } catch(RemoteException re) {
@ -3048,7 +3043,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
public void checkAccess(String src, FsAction mode) throws IOException { public void checkAccess(String src, FsAction mode) throws IOException {
checkOpen(); checkOpen();
TraceScope scope = getPathTraceScope("checkAccess", src); TraceScope scope = newPathTraceScope("checkAccess", src);
try { try {
namenode.checkAccess(src, mode); namenode.checkAccess(src, mode);
} catch (RemoteException re) { } catch (RemoteException re) {
@ -3061,12 +3056,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
} }
public DFSInotifyEventInputStream getInotifyEventStream() throws IOException { public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
return new DFSInotifyEventInputStream(traceSampler, namenode); return new DFSInotifyEventInputStream(namenode, tracer);
} }
public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid) public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid)
throws IOException { throws IOException {
return new DFSInotifyEventInputStream(traceSampler, namenode, lastReadTxid); return new DFSInotifyEventInputStream(namenode, tracer,
lastReadTxid);
} }
@Override // RemotePeerFactory @Override // RemotePeerFactory
@ -3176,28 +3172,26 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
return saslClient; return saslClient;
} }
TraceScope getPathTraceScope(String description, String path) { TraceScope newPathTraceScope(String description, String path) {
TraceScope scope = Trace.startSpan(description, traceSampler); TraceScope scope = tracer.newScope(description);
Span span = scope.getSpan(); if (path != null) {
if (span != null) { scope.addKVAnnotation("path", path);
if (path != null) {
span.addKVAnnotation("path", path);
}
} }
return scope; return scope;
} }
TraceScope getSrcDstTraceScope(String description, String src, String dst) { TraceScope newSrcDstTraceScope(String description, String src, String dst) {
TraceScope scope = Trace.startSpan(description, traceSampler); TraceScope scope = tracer.newScope(description);
Span span = scope.getSpan(); if (src != null) {
if (span != null) { scope.addKVAnnotation("src", src);
if (src != null) { }
span.addKVAnnotation("src", src); if (dst != null) {
} scope.addKVAnnotation("dst", dst);
if (dst != null) {
span.addKVAnnotation("dst", dst);
}
} }
return scope; return scope;
} }
Tracer getTracer() {
return tracer;
}
} }

View File

@ -26,9 +26,8 @@ import org.apache.hadoop.hdfs.inotify.EventBatchList;
import org.apache.hadoop.hdfs.inotify.MissingEventsException; import org.apache.hadoop.hdfs.inotify.MissingEventsException;
import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -47,11 +46,6 @@ public class DFSInotifyEventInputStream {
public static final Logger LOG = LoggerFactory.getLogger( public static final Logger LOG = LoggerFactory.getLogger(
DFSInotifyEventInputStream.class); DFSInotifyEventInputStream.class);
/**
* The trace sampler to use when making RPCs to the NameNode.
*/
private final Sampler<?> traceSampler;
private final ClientProtocol namenode; private final ClientProtocol namenode;
private Iterator<EventBatch> it; private Iterator<EventBatch> it;
private long lastReadTxid; private long lastReadTxid;
@ -65,20 +59,22 @@ public class DFSInotifyEventInputStream {
*/ */
private Random rng = new Random(); private Random rng = new Random();
private final Tracer tracer;
private static final int INITIAL_WAIT_MS = 10; private static final int INITIAL_WAIT_MS = 10;
DFSInotifyEventInputStream(Sampler<?> traceSampler, ClientProtocol namenode) DFSInotifyEventInputStream(ClientProtocol namenode, Tracer tracer)
throws IOException { throws IOException {
// Only consider new transaction IDs. // Only consider new transaction IDs.
this(traceSampler, namenode, namenode.getCurrentEditLogTxid()); this(namenode, tracer, namenode.getCurrentEditLogTxid());
} }
DFSInotifyEventInputStream(Sampler traceSampler, ClientProtocol namenode, DFSInotifyEventInputStream(ClientProtocol namenode,
long lastReadTxid) throws IOException { Tracer tracer, long lastReadTxid) throws IOException {
this.traceSampler = traceSampler;
this.namenode = namenode; this.namenode = namenode;
this.it = Iterators.emptyIterator(); this.it = Iterators.emptyIterator();
this.lastReadTxid = lastReadTxid; this.lastReadTxid = lastReadTxid;
this.tracer = tracer;
} }
/** /**
@ -98,8 +94,7 @@ public class DFSInotifyEventInputStream {
* The next available batch of events will be returned. * The next available batch of events will be returned.
*/ */
public EventBatch poll() throws IOException, MissingEventsException { public EventBatch poll() throws IOException, MissingEventsException {
TraceScope scope = TraceScope scope = tracer.newScope("inotifyPoll");
Trace.startSpan("inotifyPoll", traceSampler);
try { try {
// need to keep retrying until the NN sends us the latest committed txid // need to keep retrying until the NN sends us the latest committed txid
if (lastReadTxid == -1) { if (lastReadTxid == -1) {
@ -180,7 +175,7 @@ public class DFSInotifyEventInputStream {
*/ */
public EventBatch poll(long time, TimeUnit tu) throws IOException, public EventBatch poll(long time, TimeUnit tu) throws IOException,
InterruptedException, MissingEventsException { InterruptedException, MissingEventsException {
TraceScope scope = Trace.startSpan("inotifyPollWithTimeout", traceSampler); TraceScope scope = tracer.newScope("inotifyPollWithTimeout");
EventBatch next = null; EventBatch next = null;
try { try {
long initialTime = Time.monotonicNow(); long initialTime = Time.monotonicNow();
@ -217,7 +212,7 @@ public class DFSInotifyEventInputStream {
*/ */
public EventBatch take() throws IOException, InterruptedException, public EventBatch take() throws IOException, InterruptedException,
MissingEventsException { MissingEventsException {
TraceScope scope = Trace.startSpan("inotifyTake", traceSampler); TraceScope scope = tracer.newScope("inotifyTake");
EventBatch next = null; EventBatch next = null;
try { try {
int nextWaitMin = INITIAL_WAIT_MS; int nextWaitMin = INITIAL_WAIT_MS;

View File

@ -55,6 +55,7 @@ import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.HasEnhancedByteBufferAccess; import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
@ -78,9 +79,9 @@ import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.IdentityHashStore; import org.apache.hadoop.util.IdentityHashStore;
import org.apache.htrace.Span; import org.apache.htrace.core.SpanId;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceScope; import org.apache.htrace.core.Tracer;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -678,6 +679,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
setClientCacheContext(dfsClient.getClientContext()). setClientCacheContext(dfsClient.getClientContext()).
setUserGroupInformation(dfsClient.ugi). setUserGroupInformation(dfsClient.ugi).
setConfiguration(dfsClient.getConfiguration()). setConfiguration(dfsClient.getConfiguration()).
setTracer(dfsClient.getTracer()).
build(); build();
} }
@ -941,7 +943,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
public synchronized int read(final byte buf[], int off, int len) throws IOException { public synchronized int read(final byte buf[], int off, int len) throws IOException {
ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf); ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
TraceScope scope = TraceScope scope =
dfsClient.getPathTraceScope("DFSInputStream#byteArrayRead", src); dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src);
try { try {
return readWithStrategy(byteArrayReader, off, len); return readWithStrategy(byteArrayReader, off, len);
} finally { } finally {
@ -953,7 +955,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
public synchronized int read(final ByteBuffer buf) throws IOException { public synchronized int read(final ByteBuffer buf) throws IOException {
ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf); ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
TraceScope scope = TraceScope scope =
dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src); dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src);
try { try {
return readWithStrategy(byteBufferReader, 0, buf.remaining()); return readWithStrategy(byteBufferReader, 0, buf.remaining());
} finally { } finally {
@ -1120,14 +1122,14 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
final ByteBuffer bb, final ByteBuffer bb,
final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap, final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
final int hedgedReadId) { final int hedgedReadId) {
final Span parentSpan = Trace.currentSpan(); final SpanId parentSpanId = Tracer.getCurrentSpanId();
return new Callable<ByteBuffer>() { return new Callable<ByteBuffer>() {
@Override @Override
public ByteBuffer call() throws Exception { public ByteBuffer call() throws Exception {
byte[] buf = bb.array(); byte[] buf = bb.array();
int offset = bb.position(); int offset = bb.position();
TraceScope scope = TraceScope scope = dfsClient.getTracer().
Trace.startSpan("hedgedRead" + hedgedReadId, parentSpan); newScope("hedgedRead" + hedgedReadId, parentSpanId);
try { try {
actualGetFromOneDataNode(datanode, block, start, end, buf, actualGetFromOneDataNode(datanode, block, start, end, buf,
offset, corruptedBlockMap); offset, corruptedBlockMap);
@ -1449,8 +1451,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
@Override @Override
public int read(long position, byte[] buffer, int offset, int length) public int read(long position, byte[] buffer, int offset, int length)
throws IOException { throws IOException {
TraceScope scope = TraceScope scope = dfsClient.
dfsClient.getPathTraceScope("DFSInputStream#byteArrayPread", src); newPathTraceScope("DFSInputStream#byteArrayPread", src);
try { try {
return pread(position, buffer, offset, length); return pread(position, buffer, offset, length);
} finally { } finally {

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSOutputSummer; import org.apache.hadoop.fs.FSOutputSummer;
import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.ParentNotDirectoryException; import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Syncable; import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
@ -62,9 +63,7 @@ import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DataChecksum.Type; import org.apache.hadoop.util.DataChecksum.Type;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -227,7 +226,7 @@ public class DFSOutputStream extends FSOutputSummer
short replication, long blockSize, Progressable progress, int buffersize, short replication, long blockSize, Progressable progress, int buffersize,
DataChecksum checksum, String[] favoredNodes) throws IOException { DataChecksum checksum, String[] favoredNodes) throws IOException {
TraceScope scope = TraceScope scope =
dfsClient.getPathTraceScope("newStreamForCreate", src); dfsClient.newPathTraceScope("newStreamForCreate", src);
try { try {
HdfsFileStatus stat = null; HdfsFileStatus stat = null;
@ -350,7 +349,7 @@ public class DFSOutputStream extends FSOutputSummer
LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum, LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
String[] favoredNodes) throws IOException { String[] favoredNodes) throws IOException {
TraceScope scope = TraceScope scope =
dfsClient.getPathTraceScope("newStreamForAppend", src); dfsClient.newPathTraceScope("newStreamForAppend", src);
try { try {
final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags, final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
progress, lastBlock, stat, checksum, favoredNodes); progress, lastBlock, stat, checksum, favoredNodes);
@ -375,7 +374,7 @@ public class DFSOutputStream extends FSOutputSummer
} }
protected TraceScope createWriteTraceScope() { protected TraceScope createWriteTraceScope() {
return dfsClient.getPathTraceScope("DFSOutputStream#write", src); return dfsClient.newPathTraceScope("DFSOutputStream#write", src);
} }
// @see FSOutputSummer#writeChunk() // @see FSOutputSummer#writeChunk()
@ -495,7 +494,7 @@ public class DFSOutputStream extends FSOutputSummer
@Override @Override
public void hflush() throws IOException { public void hflush() throws IOException {
TraceScope scope = TraceScope scope =
dfsClient.getPathTraceScope("hflush", src); dfsClient.newPathTraceScope("hflush", src);
try { try {
flushOrSync(false, EnumSet.noneOf(SyncFlag.class)); flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
} finally { } finally {
@ -506,7 +505,7 @@ public class DFSOutputStream extends FSOutputSummer
@Override @Override
public void hsync() throws IOException { public void hsync() throws IOException {
TraceScope scope = TraceScope scope =
dfsClient.getPathTraceScope("hsync", src); dfsClient.newPathTraceScope("hsync", src);
try { try {
flushOrSync(true, EnumSet.noneOf(SyncFlag.class)); flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
} finally { } finally {
@ -529,7 +528,7 @@ public class DFSOutputStream extends FSOutputSummer
*/ */
public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException { public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
TraceScope scope = TraceScope scope =
dfsClient.getPathTraceScope("hsync", src); dfsClient.newPathTraceScope("hsync", src);
try { try {
flushOrSync(true, syncFlags); flushOrSync(true, syncFlags);
} finally { } finally {
@ -770,7 +769,7 @@ public class DFSOutputStream extends FSOutputSummer
@Override @Override
public synchronized void close() throws IOException { public synchronized void close() throws IOException {
TraceScope scope = TraceScope scope =
dfsClient.getPathTraceScope("DFSOutputStream#close", src); dfsClient.newPathTraceScope("DFSOutputStream#close", src);
try { try {
closeImpl(); closeImpl();
} finally { } finally {
@ -799,7 +798,7 @@ public class DFSOutputStream extends FSOutputSummer
// get last block before destroying the streamer // get last block before destroying the streamer
ExtendedBlock lastBlock = getStreamer().getBlock(); ExtendedBlock lastBlock = getStreamer().getBlock();
closeThreads(false); closeThreads(false);
TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER); TraceScope scope = dfsClient.getTracer().newScope("completeFile");
try { try {
completeFile(lastBlock); completeFile(lastBlock);
} finally { } finally {

View File

@ -27,7 +27,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
import org.apache.hadoop.hdfs.util.ByteArrayManager; import org.apache.hadoop.hdfs.util.ByteArrayManager;
import org.apache.htrace.Span; import org.apache.htrace.core.Span;
import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope;
/**************************************************************** /****************************************************************
* DFSPacket is used by DataStreamer and DFSOutputStream. * DFSPacket is used by DataStreamer and DFSOutputStream.
@ -38,7 +40,7 @@ import org.apache.htrace.Span;
@InterfaceAudience.Private @InterfaceAudience.Private
class DFSPacket { class DFSPacket {
public static final long HEART_BEAT_SEQNO = -1L; public static final long HEART_BEAT_SEQNO = -1L;
private static long[] EMPTY = new long[0]; private static SpanId[] EMPTY = new SpanId[0];
private final long seqno; // sequence number of buffer in block private final long seqno; // sequence number of buffer in block
private final long offsetInBlock; // offset in block private final long offsetInBlock; // offset in block
private boolean syncBlock; // this packet forces the current block to disk private boolean syncBlock; // this packet forces the current block to disk
@ -65,9 +67,9 @@ class DFSPacket {
private int checksumPos; private int checksumPos;
private final int dataStart; private final int dataStart;
private int dataPos; private int dataPos;
private long[] traceParents = EMPTY; private SpanId[] traceParents = EMPTY;
private int traceParentsUsed; private int traceParentsUsed;
private Span span; private TraceScope scope;
/** /**
* Create a new packet. * Create a new packet.
@ -293,7 +295,10 @@ class DFSPacket {
addTraceParent(span.getSpanId()); addTraceParent(span.getSpanId());
} }
public void addTraceParent(long id) { public void addTraceParent(SpanId id) {
if (!id.isValid()) {
return;
}
if (traceParentsUsed == traceParents.length) { if (traceParentsUsed == traceParents.length) {
int newLength = (traceParents.length == 0) ? 8 : int newLength = (traceParents.length == 0) ? 8 :
traceParents.length * 2; traceParents.length * 2;
@ -310,18 +315,18 @@ class DFSPacket {
* *
* Protected by the DFSOutputStream dataQueue lock. * Protected by the DFSOutputStream dataQueue lock.
*/ */
public long[] getTraceParents() { public SpanId[] getTraceParents() {
// Remove duplicates from the array. // Remove duplicates from the array.
int len = traceParentsUsed; int len = traceParentsUsed;
Arrays.sort(traceParents, 0, len); Arrays.sort(traceParents, 0, len);
int i = 0, j = 0; int i = 0, j = 0;
long prevVal = 0; // 0 is not a valid span id SpanId prevVal = SpanId.INVALID;
while (true) { while (true) {
if (i == len) { if (i == len) {
break; break;
} }
long val = traceParents[i]; SpanId val = traceParents[i];
if (val != prevVal) { if (!val.equals(prevVal)) {
traceParents[j] = val; traceParents[j] = val;
j++; j++;
prevVal = val; prevVal = val;
@ -335,11 +340,11 @@ class DFSPacket {
return traceParents; return traceParents;
} }
public void setTraceSpan(Span span) { public void setTraceScope(TraceScope scope) {
this.span = span; this.scope = scope;
} }
public Span getTraceSpan() { public TraceScope getTraceScope() {
return span; return scope;
} }
} }

View File

@ -41,6 +41,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf; import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@ -79,12 +80,11 @@ import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.htrace.NullScope; import org.apache.htrace.core.Sampler;
import org.apache.htrace.Sampler; import org.apache.htrace.core.Span;
import org.apache.htrace.Span; import org.apache.htrace.core.SpanId;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceInfo; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader; import com.google.common.cache.CacheLoader;
@ -505,7 +505,7 @@ class DataStreamer extends Daemon {
@Override @Override
public void run() { public void run() {
long lastPacket = Time.monotonicNow(); long lastPacket = Time.monotonicNow();
TraceScope scope = NullScope.INSTANCE; TraceScope scope = null;
while (!streamerClosed && dfsClient.clientRunning) { while (!streamerClosed && dfsClient.clientRunning) {
// if the Responder encountered an error, shutdown Responder // if the Responder encountered an error, shutdown Responder
if (errorState.hasError() && response != null) { if (errorState.hasError() && response != null) {
@ -556,12 +556,11 @@ class DataStreamer extends Daemon {
LOG.warn("Caught exception", e); LOG.warn("Caught exception", e);
} }
one = dataQueue.getFirst(); // regular data packet one = dataQueue.getFirst(); // regular data packet
long parents[] = one.getTraceParents(); SpanId[] parents = one.getTraceParents();
if (parents.length > 0) { if (parents.length > 0) {
scope = Trace.startSpan("dataStreamer", new TraceInfo(0, parents[0])); scope = dfsClient.getTracer().
// TODO: use setParents API once it's available from HTrace 3.2 newScope("dataStreamer", parents[0]);
// scope = Trace.startSpan("dataStreamer", Sampler.ALWAYS); scope.getSpan().setParents(parents);
// scope.getSpan().setParents(parents);
} }
} }
} }
@ -612,12 +611,16 @@ class DataStreamer extends Daemon {
} }
// send the packet // send the packet
Span span = null; SpanId spanId = SpanId.INVALID;
synchronized (dataQueue) { synchronized (dataQueue) {
// move packet from dataQueue to ackQueue // move packet from dataQueue to ackQueue
if (!one.isHeartbeatPacket()) { if (!one.isHeartbeatPacket()) {
span = scope.detach(); if (scope != null) {
one.setTraceSpan(span); spanId = scope.getSpanId();
scope.detach();
one.setTraceScope(scope);
}
scope = null;
dataQueue.removeFirst(); dataQueue.removeFirst();
ackQueue.addLast(one); ackQueue.addLast(one);
dataQueue.notifyAll(); dataQueue.notifyAll();
@ -630,7 +633,8 @@ class DataStreamer extends Daemon {
} }
// write out data to remote datanode // write out data to remote datanode
TraceScope writeScope = Trace.startSpan("writeTo", span); TraceScope writeScope = dfsClient.getTracer().
newScope("DataStreamer#writeTo", spanId);
try { try {
one.writeTo(blockStream); one.writeTo(blockStream);
blockStream.flush(); blockStream.flush();
@ -697,7 +701,10 @@ class DataStreamer extends Daemon {
streamerClosed = true; streamerClosed = true;
} }
} finally { } finally {
scope.close(); if (scope != null) {
scope.close();
scope = null;
}
} }
} }
closeInternal(); closeInternal();
@ -731,7 +738,8 @@ class DataStreamer extends Daemon {
* @throws IOException * @throws IOException
*/ */
void waitForAckedSeqno(long seqno) throws IOException { void waitForAckedSeqno(long seqno) throws IOException {
TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER); TraceScope scope = dfsClient.getTracer().
newScope("waitForAckedSeqno");
try { try {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Waiting for ack for: " + seqno); LOG.debug("Waiting for ack for: " + seqno);
@ -781,7 +789,7 @@ class DataStreamer extends Daemon {
while (!streamerClosed && dataQueue.size() + ackQueue.size() > while (!streamerClosed && dataQueue.size() + ackQueue.size() >
dfsClient.getConf().getWriteMaxPackets()) { dfsClient.getConf().getWriteMaxPackets()) {
if (firstWait) { if (firstWait) {
Span span = Trace.currentSpan(); Span span = Tracer.getCurrentSpan();
if (span != null) { if (span != null) {
span.addTimelineAnnotation("dataQueue.wait"); span.addTimelineAnnotation("dataQueue.wait");
} }
@ -802,7 +810,7 @@ class DataStreamer extends Daemon {
} }
} }
} finally { } finally {
Span span = Trace.currentSpan(); Span span = Tracer.getCurrentSpan();
if ((span != null) && (!firstWait)) { if ((span != null) && (!firstWait)) {
span.addTimelineAnnotation("end.wait"); span.addTimelineAnnotation("end.wait");
} }
@ -934,7 +942,7 @@ class DataStreamer extends Daemon {
setName("ResponseProcessor for block " + block); setName("ResponseProcessor for block " + block);
PipelineAck ack = new PipelineAck(); PipelineAck ack = new PipelineAck();
TraceScope scope = NullScope.INSTANCE; TraceScope scope = null;
while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) { while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
// process responses from datanodes. // process responses from datanodes.
try { try {
@ -1021,8 +1029,11 @@ class DataStreamer extends Daemon {
block.setNumBytes(one.getLastByteOffsetBlock()); block.setNumBytes(one.getLastByteOffsetBlock());
synchronized (dataQueue) { synchronized (dataQueue) {
scope = Trace.continueSpan(one.getTraceSpan()); scope = one.getTraceScope();
one.setTraceSpan(null); if (scope != null) {
scope.reattach();
one.setTraceScope(null);
}
lastAckedSeqno = seqno; lastAckedSeqno = seqno;
ackQueue.removeFirst(); ackQueue.removeFirst();
dataQueue.notifyAll(); dataQueue.notifyAll();
@ -1043,7 +1054,10 @@ class DataStreamer extends Daemon {
responderClosed = true; responderClosed = true;
} }
} finally { } finally {
if (scope != null) {
scope.close(); scope.close();
}
scope = null;
} }
} }
} }
@ -1109,11 +1123,12 @@ class DataStreamer extends Daemon {
// a client waiting on close() will be aware that the flush finished. // a client waiting on close() will be aware that the flush finished.
synchronized (dataQueue) { synchronized (dataQueue) {
DFSPacket endOfBlockPacket = dataQueue.remove(); // remove the end of block packet DFSPacket endOfBlockPacket = dataQueue.remove(); // remove the end of block packet
Span span = endOfBlockPacket.getTraceSpan(); // Close any trace span associated with this Packet
if (span != null) { TraceScope scope = endOfBlockPacket.getTraceScope();
// Close any trace span associated with this Packet if (scope != null) {
TraceScope scope = Trace.continueSpan(span); scope.reattach();
scope.close(); scope.close();
endOfBlockPacket.setTraceScope(null);
} }
assert endOfBlockPacket.isLastPacketInBlock(); assert endOfBlockPacket.isLastPacketInBlock();
assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1; assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
@ -1741,7 +1756,7 @@ class DataStreamer extends Daemon {
void queuePacket(DFSPacket packet) { void queuePacket(DFSPacket packet) {
synchronized (dataQueue) { synchronized (dataQueue) {
if (packet == null) return; if (packet == null) return;
packet.addTraceParent(Trace.currentSpan()); packet.addTraceParent(Tracer.getCurrentSpanId());
dataQueue.addLast(packet); dataQueue.addLast(packet);
lastQueuedSeqno = packet.getSeqno(); lastQueuedSeqno = packet.getSeqno();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {

View File

@ -47,9 +47,8 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -106,6 +105,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
int dataLeft = 0; int dataLeft = 0;
private final PeerCache peerCache; private final PeerCache peerCache;
private final Tracer tracer;
/* FSInputChecker interface */ /* FSInputChecker interface */
@ -210,9 +211,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
protected synchronized int readChunk(long pos, byte[] buf, int offset, protected synchronized int readChunk(long pos, byte[] buf, int offset,
int len, byte[] checksumBuf) int len, byte[] checksumBuf)
throws IOException { throws IOException {
TraceScope scope = TraceScope scope = tracer.
Trace.startSpan("RemoteBlockReader#readChunk(" + blockId + ")", newScope("RemoteBlockReader#readChunk(" + blockId + ")");
Sampler.NEVER);
try { try {
return readChunkImpl(pos, buf, offset, len, checksumBuf); return readChunkImpl(pos, buf, offset, len, checksumBuf);
} finally { } finally {
@ -346,7 +346,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
private RemoteBlockReader(String file, String bpid, long blockId, private RemoteBlockReader(String file, String bpid, long blockId,
DataInputStream in, DataChecksum checksum, boolean verifyChecksum, DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer, long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
DatanodeID datanodeID, PeerCache peerCache) { DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
// Path is used only for printing block and file information in debug // Path is used only for printing block and file information in debug
super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId + super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
":" + bpid + ":of:"+ file)/*too non path-like?*/, ":" + bpid + ":of:"+ file)/*too non path-like?*/,
@ -378,6 +378,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
bytesPerChecksum = this.checksum.getBytesPerChecksum(); bytesPerChecksum = this.checksum.getBytesPerChecksum();
checksumSize = this.checksum.getChecksumSize(); checksumSize = this.checksum.getChecksumSize();
this.peerCache = peerCache; this.peerCache = peerCache;
this.tracer = tracer;
} }
/** /**
@ -402,7 +403,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
String clientName, Peer peer, String clientName, Peer peer,
DatanodeID datanodeID, DatanodeID datanodeID,
PeerCache peerCache, PeerCache peerCache,
CachingStrategy cachingStrategy) CachingStrategy cachingStrategy,
Tracer tracer)
throws IOException { throws IOException {
// in and out will be closed when sock is closed (by the caller) // in and out will be closed when sock is closed (by the caller)
final DataOutputStream out = final DataOutputStream out =
@ -438,7 +440,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(), return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
in, checksum, verifyChecksum, startOffset, firstChunkOffset, len, in, checksum, verifyChecksum, startOffset, firstChunkOffset, len,
peer, datanodeID, peerCache); peer, datanodeID, peerCache, tracer);
} }
@Override @Override

View File

@ -48,12 +48,11 @@ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -126,6 +125,8 @@ public class RemoteBlockReader2 implements BlockReader {
private boolean sentStatusCode = false; private boolean sentStatusCode = false;
private final Tracer tracer;
@VisibleForTesting @VisibleForTesting
public Peer getPeer() { public Peer getPeer() {
return peer; return peer;
@ -144,8 +145,8 @@ public class RemoteBlockReader2 implements BlockReader {
} }
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) { if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
TraceScope scope = Trace.startSpan( TraceScope scope = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER); "RemoteBlockReader2#readNextPacket(" + blockId + ")");
try { try {
readNextPacket(); readNextPacket();
} finally { } finally {
@ -172,8 +173,8 @@ public class RemoteBlockReader2 implements BlockReader {
@Override @Override
public synchronized int read(ByteBuffer buf) throws IOException { public synchronized int read(ByteBuffer buf) throws IOException {
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) { if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
TraceScope scope = Trace.startSpan( TraceScope scope = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER); "RemoteBlockReader2#readNextPacket(" + blockId + ")");
try { try {
readNextPacket(); readNextPacket();
} finally { } finally {
@ -292,7 +293,7 @@ public class RemoteBlockReader2 implements BlockReader {
protected RemoteBlockReader2(String file, String bpid, long blockId, protected RemoteBlockReader2(String file, String bpid, long blockId,
DataChecksum checksum, boolean verifyChecksum, DataChecksum checksum, boolean verifyChecksum,
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer, long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
DatanodeID datanodeID, PeerCache peerCache) { DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
this.isLocal = DFSUtilClient.isLocalAddress(NetUtils. this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
createSocketAddr(datanodeID.getXferAddr())); createSocketAddr(datanodeID.getXferAddr()));
// Path is used only for printing block and file information in debug // Path is used only for printing block and file information in debug
@ -313,6 +314,7 @@ public class RemoteBlockReader2 implements BlockReader {
this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset); this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
bytesPerChecksum = this.checksum.getBytesPerChecksum(); bytesPerChecksum = this.checksum.getBytesPerChecksum();
checksumSize = this.checksum.getChecksumSize(); checksumSize = this.checksum.getChecksumSize();
this.tracer = tracer;
} }
@ -407,7 +409,8 @@ public class RemoteBlockReader2 implements BlockReader {
String clientName, String clientName,
Peer peer, DatanodeID datanodeID, Peer peer, DatanodeID datanodeID,
PeerCache peerCache, PeerCache peerCache,
CachingStrategy cachingStrategy) throws IOException { CachingStrategy cachingStrategy,
Tracer tracer) throws IOException {
// in and out will be closed when sock is closed (by the caller) // in and out will be closed when sock is closed (by the caller)
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream( final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
peer.getOutputStream())); peer.getOutputStream()));
@ -440,7 +443,7 @@ public class RemoteBlockReader2 implements BlockReader {
return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(), return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer, checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
datanodeID, peerCache); datanodeID, peerCache, tracer);
} }
static void checkSuccess( static void checkSuccess(

View File

@ -25,11 +25,10 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.BatchedRemoteIterator; import org.apache.hadoop.fs.BatchedRemoteIterator;
import org.apache.hadoop.fs.InvalidRequestException; import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.htrace.Sampler;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.htrace.core.TraceScope;
import org.apache.htrace.core.Tracer;
/** /**
* CacheDirectiveIterator is a remote iterator that iterates cache directives. * CacheDirectiveIterator is a remote iterator that iterates cache directives.
@ -42,14 +41,14 @@ public class CacheDirectiveIterator
private CacheDirectiveInfo filter; private CacheDirectiveInfo filter;
private final ClientProtocol namenode; private final ClientProtocol namenode;
private final Sampler<?> traceSampler; private final Tracer tracer;
public CacheDirectiveIterator(ClientProtocol namenode, public CacheDirectiveIterator(ClientProtocol namenode,
CacheDirectiveInfo filter, Sampler<?> traceSampler) { CacheDirectiveInfo filter, Tracer tracer) {
super(0L); super(0L);
this.namenode = namenode; this.namenode = namenode;
this.filter = filter; this.filter = filter;
this.traceSampler = traceSampler; this.tracer = tracer;
} }
private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) { private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) {
@ -94,7 +93,7 @@ public class CacheDirectiveIterator
public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey) public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
throws IOException { throws IOException {
BatchedEntries<CacheDirectiveEntry> entries = null; BatchedEntries<CacheDirectiveEntry> entries = null;
TraceScope scope = Trace.startSpan("listCacheDirectives", traceSampler); TraceScope scope = tracer.newScope("listCacheDirectives");
try { try {
entries = namenode.listCacheDirectives(prevKey, filter); entries = namenode.listCacheDirectives(prevKey, filter);
} catch (IOException e) { } catch (IOException e) {

View File

@ -23,9 +23,8 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.BatchedRemoteIterator; import org.apache.hadoop.fs.BatchedRemoteIterator;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
/** /**
* CachePoolIterator is a remote iterator that iterates cache pools. * CachePoolIterator is a remote iterator that iterates cache pools.
@ -37,18 +36,18 @@ public class CachePoolIterator
extends BatchedRemoteIterator<String, CachePoolEntry> { extends BatchedRemoteIterator<String, CachePoolEntry> {
private final ClientProtocol namenode; private final ClientProtocol namenode;
private final Sampler traceSampler; private final Tracer tracer;
public CachePoolIterator(ClientProtocol namenode, Sampler traceSampler) { public CachePoolIterator(ClientProtocol namenode, Tracer tracer) {
super(""); super("");
this.namenode = namenode; this.namenode = namenode;
this.traceSampler = traceSampler; this.tracer = tracer;
} }
@Override @Override
public BatchedEntries<CachePoolEntry> makeRequest(String prevKey) public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
throws IOException { throws IOException {
TraceScope scope = Trace.startSpan("listCachePools", traceSampler); TraceScope scope = tracer.newScope("listCachePools");
try { try {
return namenode.listCachePools(prevKey); return namenode.listCachePools(prevKey);
} finally { } finally {

View File

@ -23,9 +23,8 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.BatchedRemoteIterator; import org.apache.hadoop.fs.BatchedRemoteIterator;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
/** /**
* EncryptionZoneIterator is a remote iterator that iterates over encryption * EncryptionZoneIterator is a remote iterator that iterates over encryption
@ -37,19 +36,18 @@ public class EncryptionZoneIterator
extends BatchedRemoteIterator<Long, EncryptionZone> { extends BatchedRemoteIterator<Long, EncryptionZone> {
private final ClientProtocol namenode; private final ClientProtocol namenode;
private final Sampler<?> traceSampler; private final Tracer tracer;
public EncryptionZoneIterator(ClientProtocol namenode, public EncryptionZoneIterator(ClientProtocol namenode, Tracer tracer) {
Sampler<?> traceSampler) {
super(Long.valueOf(0)); super(Long.valueOf(0));
this.namenode = namenode; this.namenode = namenode;
this.traceSampler = traceSampler; this.tracer = tracer;
} }
@Override @Override
public BatchedEntries<EncryptionZone> makeRequest(Long prevId) public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
throws IOException { throws IOException {
TraceScope scope = Trace.startSpan("listEncryptionZones", traceSampler); TraceScope scope = tracer.newScope("listEncryptionZones");
try { try {
return namenode.listEncryptionZones(prevId); return namenode.listEncryptionZones(prevId);
} finally { } finally {

View File

@ -35,10 +35,8 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.htrace.Span; import org.apache.htrace.core.SpanId;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceInfo;
import org.apache.htrace.TraceScope;
/** /**
* Static utilities for dealing with the protocol buffers used by the * Static utilities for dealing with the protocol buffers used by the
@ -89,39 +87,21 @@ public abstract class DataTransferProtoUtil {
BaseHeaderProto.Builder builder = BaseHeaderProto.newBuilder() BaseHeaderProto.Builder builder = BaseHeaderProto.newBuilder()
.setBlock(PBHelperClient.convert(blk)) .setBlock(PBHelperClient.convert(blk))
.setToken(PBHelperClient.convert(blockToken)); .setToken(PBHelperClient.convert(blockToken));
if (Trace.isTracing()) { SpanId spanId = Tracer.getCurrentSpanId();
Span s = Trace.currentSpan(); if (spanId.isValid()) {
builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder() builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder()
.setTraceId(s.getTraceId()) .setTraceId(spanId.getHigh())
.setParentId(s.getSpanId())); .setParentId(spanId.getLow()));
} }
return builder.build(); return builder.build();
} }
public static TraceInfo fromProto(DataTransferTraceInfoProto proto) { public static SpanId fromProto(DataTransferTraceInfoProto proto) {
if (proto == null) return null; if ((proto != null) && proto.hasTraceId() &&
if (!proto.hasTraceId()) return null; proto.hasParentId()) {
return new TraceInfo(proto.getTraceId(), proto.getParentId()); return new SpanId(proto.getTraceId(), proto.getParentId());
}
public static TraceScope continueTraceSpan(ClientOperationHeaderProto header,
String description) {
return continueTraceSpan(header.getBaseHeader(), description);
}
public static TraceScope continueTraceSpan(BaseHeaderProto header,
String description) {
return continueTraceSpan(header.getTraceInfo(), description);
}
public static TraceScope continueTraceSpan(DataTransferTraceInfoProto proto,
String description) {
TraceScope scope = null;
TraceInfo info = fromProto(proto);
if (info != null) {
scope = Trace.startSpan(description, info);
} }
return scope; return null;
} }
public static void checkBlockOpStatus( public static void checkBlockOpStatus(

View File

@ -48,8 +48,8 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.htrace.Trace; import org.apache.htrace.core.SpanId;
import org.apache.htrace.Span; import org.apache.htrace.core.Tracer;
import com.google.protobuf.Message; import com.google.protobuf.Message;
@ -200,10 +200,11 @@ public class Sender implements DataTransferProtocol {
ReleaseShortCircuitAccessRequestProto.Builder builder = ReleaseShortCircuitAccessRequestProto.Builder builder =
ReleaseShortCircuitAccessRequestProto.newBuilder(). ReleaseShortCircuitAccessRequestProto.newBuilder().
setSlotId(PBHelperClient.convert(slotId)); setSlotId(PBHelperClient.convert(slotId));
if (Trace.isTracing()) { SpanId spanId = Tracer.getCurrentSpanId();
Span s = Trace.currentSpan(); if (spanId.isValid()) {
builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder() builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
.setTraceId(s.getTraceId()).setParentId(s.getSpanId())); setTraceId(spanId.getHigh()).
setParentId(spanId.getLow()));
} }
ReleaseShortCircuitAccessRequestProto proto = builder.build(); ReleaseShortCircuitAccessRequestProto proto = builder.build();
send(out, Op.RELEASE_SHORT_CIRCUIT_FDS, proto); send(out, Op.RELEASE_SHORT_CIRCUIT_FDS, proto);
@ -214,10 +215,11 @@ public class Sender implements DataTransferProtocol {
ShortCircuitShmRequestProto.Builder builder = ShortCircuitShmRequestProto.Builder builder =
ShortCircuitShmRequestProto.newBuilder(). ShortCircuitShmRequestProto.newBuilder().
setClientName(clientName); setClientName(clientName);
if (Trace.isTracing()) { SpanId spanId = Tracer.getCurrentSpanId();
Span s = Trace.currentSpan(); if (spanId.isValid()) {
builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder() builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
.setTraceId(s.getTraceId()).setParentId(s.getSpanId())); setTraceId(spanId.getHigh()).
setParentId(spanId.getLow()));
} }
ShortCircuitShmRequestProto proto = builder.build(); ShortCircuitShmRequestProto proto = builder.build();
send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto); send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto);

View File

@ -638,6 +638,8 @@ Release 2.8.0 - UNRELEASED
HDFS-8740. Move DistributedFileSystem to hadoop-hdfs-client. (Mingliang Liu HDFS-8740. Move DistributedFileSystem to hadoop-hdfs-client. (Mingliang Liu
via wheat9) via wheat9)
HDFS-9080. Update htrace version to 4.0.1 (cmccabe)
OPTIMIZATIONS OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

View File

@ -192,7 +192,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.htrace</groupId> <groupId>org.apache.htrace</groupId>
<artifactId>htrace-core</artifactId> <artifactId>htrace-core4</artifactId>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>

View File

@ -64,12 +64,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT = public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT; HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
// HDFS HTrace configuration is controlled by dfs.htrace.spanreceiver.classes, public static final String DFS_CLIENT_HTRACE_SAMPLER_CLASSES =
// etc. "dfs.client.htrace.sampler.classes";
public static final String DFS_SERVER_HTRACE_PREFIX = "dfs.htrace.";
@Deprecated
public static final String DFS_CLIENT_HTRACE_PREFIX =
HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX;
// HA related configuration // HA related configuration
public static final String DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration"; public static final String DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration";

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hdfs.protocol.datatransfer; package org.apache.hadoop.hdfs.protocol.datatransfer;
import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto; import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto;
import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.continueTraceSpan;
import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed; import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
import java.io.DataInputStream; import java.io.DataInputStream;
@ -27,7 +26,10 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; 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.OpCopyBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
@ -40,14 +42,21 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmR
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
import org.apache.htrace.TraceScope; import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope;
import org.apache.htrace.core.Tracer;
/** Receiver */ /** Receiver */
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
public abstract class Receiver implements DataTransferProtocol { public abstract class Receiver implements DataTransferProtocol {
private final Tracer tracer;
protected DataInputStream in; protected DataInputStream in;
protected Receiver(Tracer tracer) {
this.tracer = tracer;
}
/** Initialize a receiver for DataTransferProtocol with a socket. */ /** Initialize a receiver for DataTransferProtocol with a socket. */
protected void initialize(final DataInputStream in) { protected void initialize(final DataInputStream in) {
this.in = in; this.in = in;
@ -64,6 +73,26 @@ public abstract class Receiver implements DataTransferProtocol {
return Op.read(in); return Op.read(in);
} }
private TraceScope continueTraceSpan(DataTransferTraceInfoProto proto,
String description) {
TraceScope scope = null;
SpanId spanId = fromProto(proto);
if (spanId != null) {
scope = tracer.newScope(description, spanId);
}
return scope;
}
private TraceScope continueTraceSpan(ClientOperationHeaderProto header,
String description) {
return continueTraceSpan(header.getBaseHeader(), description);
}
private TraceScope continueTraceSpan(BaseHeaderProto header,
String description) {
return continueTraceSpan(header.getTraceInfo(), description);
}
/** Process op by the corresponding method. */ /** Process op by the corresponding method. */
protected final void processOp(Op op) throws IOException { protected final void processOp(Op op) throws IOException {
switch(op) { switch(op) {

View File

@ -43,10 +43,12 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.source.JvmMetrics; import org.apache.hadoop.metrics2.source.JvmMetrics;
import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.tracing.TraceUtils;
import org.apache.hadoop.util.DiskChecker; import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.apache.htrace.core.Tracer;
import org.mortbay.util.ajax.JSON; import org.mortbay.util.ajax.JSON;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -69,6 +71,7 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
private ObjectName journalNodeInfoBeanName; private ObjectName journalNodeInfoBeanName;
private String httpServerURI; private String httpServerURI;
private File localDir; private File localDir;
Tracer tracer;
static { static {
HdfsConfiguration.init(); HdfsConfiguration.init();
@ -105,6 +108,11 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
this.localDir = new File( this.localDir = new File(
conf.get(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, conf.get(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY,
DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_DEFAULT).trim()); DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_DEFAULT).trim());
if (this.tracer == null) {
this.tracer = new Tracer.Builder("JournalNode").
conf(TraceUtils.wrapHadoopConf("journalnode.htrace", conf)).
build();
}
} }
private static void validateAndCreateJournalDir(File dir) throws IOException { private static void validateAndCreateJournalDir(File dir) throws IOException {
@ -203,6 +211,10 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
MBeans.unregister(journalNodeInfoBeanName); MBeans.unregister(journalNodeInfoBeanName);
journalNodeInfoBeanName = null; journalNodeInfoBeanName = null;
} }
if (tracer != null) {
tracer.close();
tracer = null;
}
} }
/** /**
@ -326,5 +338,4 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
public Long getJournalCTime(String journalId) throws IOException { public Long getJournalCTime(String journalId) throws IOException {
return getOrCreateJournal(journalId).getJournalCTime(); return getOrCreateJournal(journalId).getJournalCTime();
} }
} }

View File

@ -85,6 +85,7 @@ class JournalNodeRpcServer implements QJournalProtocol {
CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) { CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
server.refreshServiceAcl(confCopy, new HDFSPolicyProvider()); server.refreshServiceAcl(confCopy, new HDFSPolicyProvider());
} }
this.server.setTracer(jn.tracer);
} }
void start() { void start() {

View File

@ -47,9 +47,8 @@ import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.net.SocketOutputStream; import org.apache.hadoop.net.SocketOutputStream;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
import org.apache.htrace.Sampler; import org.apache.htrace.core.Sampler;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceScope;
import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED; import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED;
import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_SEQUENTIAL; import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_SEQUENTIAL;
@ -708,8 +707,8 @@ class BlockSender implements java.io.Closeable {
*/ */
long sendBlock(DataOutputStream out, OutputStream baseStream, long sendBlock(DataOutputStream out, OutputStream baseStream,
DataTransferThrottler throttler) throws IOException { DataTransferThrottler throttler) throws IOException {
TraceScope scope = TraceScope scope = datanode.tracer.
Trace.startSpan("sendBlock_" + block.getBlockId(), Sampler.NEVER); newScope("sendBlock_" + block.getBlockId());
try { try {
return doSendBlock(out, baseStream, throttler); return doSendBlock(out, baseStream, throttler);
} finally { } finally {

View File

@ -189,12 +189,13 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.tracing.SpanReceiverHost;
import org.apache.hadoop.tracing.SpanReceiverInfo; import org.apache.hadoop.tracing.SpanReceiverInfo;
import org.apache.hadoop.tracing.TraceAdminPB.TraceAdminService; import org.apache.hadoop.tracing.TraceAdminPB.TraceAdminService;
import org.apache.hadoop.tracing.TraceAdminProtocol; import org.apache.hadoop.tracing.TraceAdminProtocol;
import org.apache.hadoop.tracing.TraceAdminProtocolPB; import org.apache.hadoop.tracing.TraceAdminProtocolPB;
import org.apache.hadoop.tracing.TraceAdminProtocolServerSideTranslatorPB; import org.apache.hadoop.tracing.TraceAdminProtocolServerSideTranslatorPB;
import org.apache.hadoop.tracing.TraceUtils;
import org.apache.hadoop.tracing.TracerConfigurationManager;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DiskChecker; import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.util.DiskChecker.DiskErrorException; import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@ -204,6 +205,7 @@ import org.apache.hadoop.util.ServicePlugin;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.util.VersionInfo;
import org.apache.htrace.core.Tracer;
import org.mortbay.util.ajax.JSON; import org.mortbay.util.ajax.JSON;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -288,6 +290,8 @@ public class DataNode extends ReconfigurableBase
public static final Log METRICS_LOG = LogFactory.getLog("DataNodeMetricsLog"); public static final Log METRICS_LOG = LogFactory.getLog("DataNodeMetricsLog");
private static final String DATANODE_HTRACE_PREFIX = "datanode.htrace.";
/** /**
* Use {@link NetUtils#createSocketAddr(String)} instead. * Use {@link NetUtils#createSocketAddr(String)} instead.
*/ */
@ -368,11 +372,18 @@ public class DataNode extends ReconfigurableBase
private boolean isPermissionEnabled; private boolean isPermissionEnabled;
private String dnUserName = null; private String dnUserName = null;
private SpanReceiverHost spanReceiverHost; final Tracer tracer;
private final TracerConfigurationManager tracerConfigurationManager;
private static final int NUM_CORES = Runtime.getRuntime() private static final int NUM_CORES = Runtime.getRuntime()
.availableProcessors(); .availableProcessors();
private static final double CONGESTION_RATIO = 1.5; private static final double CONGESTION_RATIO = 1.5;
private static Tracer createTracer(Configuration conf) {
return new Tracer.Builder("DataNode").
conf(TraceUtils.wrapHadoopConf(DATANODE_HTRACE_PREFIX , conf)).
build();
}
private long[] oobTimeouts; /** timeout value of each OOB type */ private long[] oobTimeouts; /** timeout value of each OOB type */
private ScheduledThreadPoolExecutor metricsLoggerTimer; private ScheduledThreadPoolExecutor metricsLoggerTimer;
@ -385,6 +396,9 @@ public class DataNode extends ReconfigurableBase
DataNode(final Configuration conf) { DataNode(final Configuration conf) {
super(conf); super(conf);
this.blockScanner = new BlockScanner(this, conf); this.blockScanner = new BlockScanner(this, conf);
this.tracer = createTracer(conf);
this.tracerConfigurationManager =
new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
this.fileDescriptorPassingDisabledReason = null; this.fileDescriptorPassingDisabledReason = null;
this.maxNumberOfBlocksToLog = 0; this.maxNumberOfBlocksToLog = 0;
this.confVersion = null; this.confVersion = null;
@ -405,6 +419,9 @@ public class DataNode extends ReconfigurableBase
final List<StorageLocation> dataDirs, final List<StorageLocation> dataDirs,
final SecureResources resources) throws IOException { final SecureResources resources) throws IOException {
super(conf); super(conf);
this.tracer = createTracer(conf);
this.tracerConfigurationManager =
new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
this.blockScanner = new BlockScanner(this, conf); this.blockScanner = new BlockScanner(this, conf);
this.lastDiskErrorCheck = 0; this.lastDiskErrorCheck = 0;
this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY, this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
@ -1144,9 +1161,6 @@ public class DataNode extends ReconfigurableBase
this.dnConf = new DNConf(conf); this.dnConf = new DNConf(conf);
checkSecureConfig(dnConf, conf, resources); checkSecureConfig(dnConf, conf, resources);
this.spanReceiverHost =
SpanReceiverHost.get(conf, DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX);
if (dnConf.maxLockedMemory > 0) { if (dnConf.maxLockedMemory > 0) {
if (!NativeIO.POSIX.getCacheManipulator().verifyCanMlock()) { if (!NativeIO.POSIX.getCacheManipulator().verifyCanMlock()) {
throw new RuntimeException(String.format( throw new RuntimeException(String.format(
@ -1851,9 +1865,6 @@ public class DataNode extends ReconfigurableBase
MBeans.unregister(dataNodeInfoBeanName); MBeans.unregister(dataNodeInfoBeanName);
dataNodeInfoBeanName = null; dataNodeInfoBeanName = null;
} }
if (this.spanReceiverHost != null) {
this.spanReceiverHost.closeReceivers();
}
if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown(); if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown();
LOG.info("Shutdown complete."); LOG.info("Shutdown complete.");
synchronized(this) { synchronized(this) {
@ -1862,6 +1873,7 @@ public class DataNode extends ReconfigurableBase
// Notify the main thread. // Notify the main thread.
notifyAll(); notifyAll();
} }
tracer.close();
} }
@ -2294,6 +2306,7 @@ public class DataNode extends ReconfigurableBase
if (localDataXceiverServer != null) { if (localDataXceiverServer != null) {
localDataXceiverServer.start(); localDataXceiverServer.start();
} }
ipcServer.setTracer(tracer);
ipcServer.start(); ipcServer.start();
startPlugins(conf); startPlugins(conf);
} }
@ -3283,19 +3296,19 @@ public class DataNode extends ReconfigurableBase
@Override @Override
public SpanReceiverInfo[] listSpanReceivers() throws IOException { public SpanReceiverInfo[] listSpanReceivers() throws IOException {
checkSuperuserPrivilege(); checkSuperuserPrivilege();
return spanReceiverHost.listSpanReceivers(); return tracerConfigurationManager.listSpanReceivers();
} }
@Override @Override
public long addSpanReceiver(SpanReceiverInfo info) throws IOException { public long addSpanReceiver(SpanReceiverInfo info) throws IOException {
checkSuperuserPrivilege(); checkSuperuserPrivilege();
return spanReceiverHost.addSpanReceiver(info); return tracerConfigurationManager.addSpanReceiver(info);
} }
@Override @Override
public void removeSpanReceiver(long id) throws IOException { public void removeSpanReceiver(long id) throws IOException {
checkSuperuserPrivilege(); checkSuperuserPrivilege();
spanReceiverHost.removeSpanReceiver(id); tracerConfigurationManager.removeSpanReceiver(id);
} }
/** /**

View File

@ -126,7 +126,7 @@ class DataXceiver extends Receiver implements Runnable {
private DataXceiver(Peer peer, DataNode datanode, private DataXceiver(Peer peer, DataNode datanode,
DataXceiverServer dataXceiverServer) throws IOException { DataXceiverServer dataXceiverServer) throws IOException {
super(datanode.tracer);
this.peer = peer; this.peer = peer;
this.dnConf = datanode.getDnConf(); this.dnConf = datanode.getDnConf();
this.socketIn = peer.getInputStream(); this.socketIn = peer.getInputStream();

View File

@ -71,12 +71,14 @@ import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol; import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
import org.apache.hadoop.tools.GetUserMappingsProtocol; import org.apache.hadoop.tools.GetUserMappingsProtocol;
import org.apache.hadoop.tracing.SpanReceiverHost;
import org.apache.hadoop.tracing.TraceAdminProtocol; import org.apache.hadoop.tracing.TraceAdminProtocol;
import org.apache.hadoop.tracing.TraceUtils;
import org.apache.hadoop.tracing.TracerConfigurationManager;
import org.apache.hadoop.util.ExitUtil.ExitException; import org.apache.hadoop.util.ExitUtil.ExitException;
import org.apache.hadoop.util.JvmPauseMonitor; import org.apache.hadoop.util.JvmPauseMonitor;
import org.apache.hadoop.util.ServicePlugin; import org.apache.hadoop.util.ServicePlugin;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -318,6 +320,8 @@ public class NameNode implements NameNodeStatusMXBean {
public static final HAState ACTIVE_STATE = new ActiveState(); public static final HAState ACTIVE_STATE = new ActiveState();
public static final HAState STANDBY_STATE = new StandbyState(); public static final HAState STANDBY_STATE = new StandbyState();
private static final String NAMENODE_HTRACE_PREFIX = "namenode.htrace.";
public static final Log MetricsLog = public static final Log MetricsLog =
LogFactory.getLog("NameNodeMetricsLog"); LogFactory.getLog("NameNodeMetricsLog");
@ -345,7 +349,8 @@ public class NameNode implements NameNodeStatusMXBean {
private JvmPauseMonitor pauseMonitor; private JvmPauseMonitor pauseMonitor;
private ObjectName nameNodeStatusBeanName; private ObjectName nameNodeStatusBeanName;
SpanReceiverHost spanReceiverHost; protected final Tracer tracer;
protected final TracerConfigurationManager tracerConfigurationManager;
ScheduledThreadPoolExecutor metricsLoggerTimer; ScheduledThreadPoolExecutor metricsLoggerTimer;
/** /**
@ -620,9 +625,6 @@ public class NameNode implements NameNodeStatusMXBean {
startHttpServer(conf); startHttpServer(conf);
} }
this.spanReceiverHost =
SpanReceiverHost.get(conf, DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX);
loadNamesystem(conf); loadNamesystem(conf);
rpcServer = createRpcServer(conf); rpcServer = createRpcServer(conf);
@ -812,8 +814,13 @@ public class NameNode implements NameNodeStatusMXBean {
this(conf, NamenodeRole.NAMENODE); this(conf, NamenodeRole.NAMENODE);
} }
protected NameNode(Configuration conf, NamenodeRole role) protected NameNode(Configuration conf, NamenodeRole role)
throws IOException { throws IOException {
this.tracer = new Tracer.Builder("NameNode").
conf(TraceUtils.wrapHadoopConf(NAMENODE_HTRACE_PREFIX, conf)).
build();
this.tracerConfigurationManager =
new TracerConfigurationManager(NAMENODE_HTRACE_PREFIX, conf);
this.conf = conf; this.conf = conf;
this.role = role; this.role = role;
setClientNamenodeAddress(conf); setClientNamenodeAddress(conf);
@ -896,10 +903,8 @@ public class NameNode implements NameNodeStatusMXBean {
MBeans.unregister(nameNodeStatusBeanName); MBeans.unregister(nameNodeStatusBeanName);
nameNodeStatusBeanName = null; nameNodeStatusBeanName = null;
} }
if (this.spanReceiverHost != null) {
this.spanReceiverHost.closeReceivers();
}
} }
tracer.close();
} }
synchronized boolean isStopRequested() { synchronized boolean isStopRequested() {

View File

@ -412,6 +412,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
FSLimitException.PathComponentTooLongException.class, FSLimitException.PathComponentTooLongException.class,
FSLimitException.MaxDirectoryItemsExceededException.class, FSLimitException.MaxDirectoryItemsExceededException.class,
UnresolvedPathException.class); UnresolvedPathException.class);
clientRpcServer.setTracer(nn.tracer);
if (serviceRpcServer != null) {
clientRpcServer.setTracer(nn.tracer);
}
} }
/** Allow access to the client RPC server for testing */ /** Allow access to the client RPC server for testing */
@ -2019,20 +2023,20 @@ class NameNodeRpcServer implements NamenodeProtocols {
public SpanReceiverInfo[] listSpanReceivers() throws IOException { public SpanReceiverInfo[] listSpanReceivers() throws IOException {
checkNNStartup(); checkNNStartup();
namesystem.checkSuperuserPrivilege(); namesystem.checkSuperuserPrivilege();
return nn.spanReceiverHost.listSpanReceivers(); return nn.tracerConfigurationManager.listSpanReceivers();
} }
@Override // TraceAdminProtocol @Override // TraceAdminProtocol
public long addSpanReceiver(SpanReceiverInfo info) throws IOException { public long addSpanReceiver(SpanReceiverInfo info) throws IOException {
checkNNStartup(); checkNNStartup();
namesystem.checkSuperuserPrivilege(); namesystem.checkSuperuserPrivilege();
return nn.spanReceiverHost.addSpanReceiver(info); return nn.tracerConfigurationManager.addSpanReceiver(info);
} }
@Override // TraceAdminProtocol @Override // TraceAdminProtocol
public void removeSpanReceiver(long id) throws IOException { public void removeSpanReceiver(long id) throws IOException {
checkNNStartup(); checkNNStartup();
namesystem.checkSuperuserPrivilege(); namesystem.checkSuperuserPrivilege();
nn.spanReceiverHost.removeSpanReceiver(id); nn.tracerConfigurationManager.removeSpanReceiver(id);
} }
} }

View File

@ -81,6 +81,7 @@ import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.htrace.core.Tracer;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -136,6 +137,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
private boolean showReplicaDetails = false; private boolean showReplicaDetails = false;
private long staleInterval; private long staleInterval;
private Tracer tracer;
/** /**
* True if we encountered an internal error during FSCK, such as not being * True if we encountered an internal error during FSCK, such as not being
* able to delete a corrupt file. * able to delete a corrupt file.
@ -198,6 +201,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
this.staleInterval = this.staleInterval =
conf.getLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY, conf.getLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT); DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);
this.tracer = new Tracer.Builder("NamenodeFsck").build();
for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) { for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) {
String key = it.next(); String key = it.next();
@ -850,6 +854,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
setCachingStrategy(CachingStrategy.newDropBehind()). setCachingStrategy(CachingStrategy.newDropBehind()).
setClientCacheContext(dfs.getClientContext()). setClientCacheContext(dfs.getClientContext()).
setConfiguration(namenode.conf). setConfiguration(namenode.conf).
setTracer(tracer).
setRemotePeerFactory(new RemotePeerFactory() { setRemotePeerFactory(new RemotePeerFactory() {
@Override @Override
public Peer newConnectedPeer(InetSocketAddress addr, public Peer newConnectedPeer(InetSocketAddress addr,

View File

@ -2440,19 +2440,10 @@
</property> </property>
<property> <property>
<name>dfs.htrace.spanreceiver.classes</name> <name>dfs.client.htrace.sampler.classes</name>
<value></value> <value></value>
<description> <description>
The class name of the HTrace SpanReceiver for the NameNode and DataNode. The class names of the HTrace Samplers to use for the HDFS client.
</description>
</property>
<property>
<name>dfs.client.htrace.spanreceiver.classes</name>
<value></value>
<description>
The class name of the HTrace SpanReceiver for the HDFS client. You do not
need to enable this if your client program has been modified to use HTrace.
</description> </description>
</property> </property>

View File

@ -30,6 +30,7 @@ import java.util.Random;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
@ -192,6 +193,7 @@ public class BlockReaderTestUtil {
setCachingStrategy(CachingStrategy.newDefaultStrategy()). setCachingStrategy(CachingStrategy.newDefaultStrategy()).
setConfiguration(fs.getConf()). setConfiguration(fs.getConf()).
setAllowShortCircuitLocalReads(true). setAllowShortCircuitLocalReads(true).
setTracer(FsTracer.get(fs.getConf())).
setRemotePeerFactory(new RemotePeerFactory() { setRemotePeerFactory(new RemotePeerFactory() {
@Override @Override
public Peer newConnectedPeer(InetSocketAddress addr, public Peer newConnectedPeer(InetSocketAddress addr,

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory; import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@ -195,6 +196,7 @@ public class TestBlockReaderLocal {
setShortCircuitReplica(replica). setShortCircuitReplica(replica).
setCachingStrategy(new CachingStrategy(false, readahead)). setCachingStrategy(new CachingStrategy(false, readahead)).
setVerifyChecksum(checksum). setVerifyChecksum(checksum).
setTracer(FsTracer.get(conf)).
build(); build();
dataIn = null; dataIn = null;
metaIn = null; metaIn = null;

View File

@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DataStreamer.LastExceptionInStreamer; import org.apache.hadoop.hdfs.DataStreamer.LastExceptionInStreamer;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf; import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@ -34,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.htrace.core.SpanId;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -119,6 +121,7 @@ public class TestDFSOutputStream {
DfsClientConf dfsClientConf = mock(DfsClientConf.class); DfsClientConf dfsClientConf = mock(DfsClientConf.class);
DFSClient client = mock(DFSClient.class); DFSClient client = mock(DFSClient.class);
when(client.getConf()).thenReturn(dfsClientConf); when(client.getConf()).thenReturn(dfsClientConf);
when(client.getTracer()).thenReturn(FsTracer.get(new Configuration()));
client.clientRunning = true; client.clientRunning = true;
DataStreamer stream = new DataStreamer( DataStreamer stream = new DataStreamer(
mock(HdfsFileStatus.class), mock(HdfsFileStatus.class),
@ -139,7 +142,7 @@ public class TestDFSOutputStream {
Whitebox.getInternalState(stream, "congestedNodes"); Whitebox.getInternalState(stream, "congestedNodes");
congestedNodes.add(mock(DatanodeInfo.class)); congestedNodes.add(mock(DatanodeInfo.class));
DFSPacket packet = mock(DFSPacket.class); DFSPacket packet = mock(DFSPacket.class);
when(packet.getTraceParents()).thenReturn(new long[] {}); when(packet.getTraceParents()).thenReturn(new SpanId[] {});
dataQueue.add(packet); dataQueue.add(packet);
stream.run(); stream.run();
Assert.assertTrue(congestedNodes.isEmpty()); Assert.assertTrue(congestedNodes.isEmpty());

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
import java.util.Random; import java.util.Random;
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.htrace.core.SpanId;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -70,24 +71,24 @@ public class TestDFSPacket {
public void testAddParentsGetParents() throws Exception { public void testAddParentsGetParents() throws Exception {
DFSPacket p = new DFSPacket(null, maxChunksPerPacket, DFSPacket p = new DFSPacket(null, maxChunksPerPacket,
0, 0, checksumSize, false); 0, 0, checksumSize, false);
long parents[] = p.getTraceParents(); SpanId parents[] = p.getTraceParents();
Assert.assertEquals(0, parents.length); Assert.assertEquals(0, parents.length);
p.addTraceParent(123); p.addTraceParent(new SpanId(0, 123));
p.addTraceParent(123); p.addTraceParent(new SpanId(0, 123));
parents = p.getTraceParents(); parents = p.getTraceParents();
Assert.assertEquals(1, parents.length); Assert.assertEquals(1, parents.length);
Assert.assertEquals(123, parents[0]); Assert.assertEquals(new SpanId(0, 123), parents[0]);
parents = p.getTraceParents(); // test calling 'get' again. parents = p.getTraceParents(); // test calling 'get' again.
Assert.assertEquals(1, parents.length); Assert.assertEquals(1, parents.length);
Assert.assertEquals(123, parents[0]); Assert.assertEquals(new SpanId(0, 123), parents[0]);
p.addTraceParent(1); p.addTraceParent(new SpanId(0, 1));
p.addTraceParent(456); p.addTraceParent(new SpanId(0, 456));
p.addTraceParent(789); p.addTraceParent(new SpanId(0, 789));
parents = p.getTraceParents(); parents = p.getTraceParents();
Assert.assertEquals(4, parents.length); Assert.assertEquals(4, parents.length);
Assert.assertEquals(1, parents[0]); Assert.assertEquals(new SpanId(0, 1), parents[0]);
Assert.assertEquals(123, parents[1]); Assert.assertEquals(new SpanId(0, 123), parents[1]);
Assert.assertEquals(456, parents[2]); Assert.assertEquals(new SpanId(0, 456), parents[2]);
Assert.assertEquals(789, parents[3]); Assert.assertEquals(new SpanId(0, 789), parents[3]);
} }
} }

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.fs.CacheFlag;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystemTestHelper; import org.apache.hadoop.fs.FileSystemTestHelper;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.fs.InvalidRequestException; import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.RemoteIterator;
@ -89,7 +90,6 @@ import org.apache.hadoop.util.GSet;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.htrace.Sampler;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -910,7 +910,7 @@ public class TestCacheDirectives {
// Uncache and check each path in sequence // Uncache and check each path in sequence
RemoteIterator<CacheDirectiveEntry> entries = RemoteIterator<CacheDirectiveEntry> entries =
new CacheDirectiveIterator(nnRpc, null, Sampler.NEVER); new CacheDirectiveIterator(nnRpc, null, FsTracer.get(conf));
for (int i=0; i<numFiles; i++) { for (int i=0; i<numFiles; i++) {
CacheDirectiveEntry entry = entries.next(); CacheDirectiveEntry entry = entries.next();
nnRpc.removeCacheDirective(entry.getInfo().getId()); nnRpc.removeCacheDirective(entry.getInfo().getId());

View File

@ -387,6 +387,7 @@ public class TestFsck {
// Copy the non-corrupt blocks of corruptFileName to lost+found. // Copy the non-corrupt blocks of corruptFileName to lost+found.
outStr = runFsck(conf, 1, false, "/", "-move"); outStr = runFsck(conf, 1, false, "/", "-move");
FSImage.LOG.info("WATERMELON: outStr = " + outStr);
assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS)); assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
// Make sure that we properly copied the block files from the DataNodes // Make sure that we properly copied the block files from the DataNodes

View File

@ -77,8 +77,8 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
// Some properties have moved to HdfsClientConfigKeys // Some properties have moved to HdfsClientConfigKeys
xmlPropsToSkipCompare.add("dfs.client.short.circuit.replica.stale.threshold.ms"); xmlPropsToSkipCompare.add("dfs.client.short.circuit.replica.stale.threshold.ms");
// Ignore SpanReceiveHost properties // Ignore HTrace properties
xmlPropsToSkipCompare.add("dfs.htrace.spanreceiver.classes"); xmlPropsToSkipCompare.add("fs.client.htrace");
xmlPropsToSkipCompare.add("dfs.client.htrace.spanreceiver.classes"); xmlPropsToSkipCompare.add("hadoop.htrace");
} }
} }

View File

@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.net.unix.TemporarySocketDirectory; import org.apache.hadoop.net.unix.TemporarySocketDirectory;
import org.apache.htrace.core.Tracer;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -58,8 +59,8 @@ public class TestTraceAdmin {
public void testCreateAndDestroySpanReceiver() throws Exception { public void testCreateAndDestroySpanReceiver() throws Exception {
Configuration conf = new Configuration(); Configuration conf = new Configuration();
conf = new Configuration(); conf = new Configuration();
conf.set(DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX + conf.set(TraceUtils.DEFAULT_HADOOP_PREFIX +
SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX, ""); Tracer.SPAN_RECEIVER_CLASSES_KEY, "");
MiniDFSCluster cluster = MiniDFSCluster cluster =
new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
cluster.waitActive(); cluster.waitActive();
@ -74,12 +75,12 @@ public class TestTraceAdmin {
Assert.assertEquals("ret:0, Added trace span receiver 1 with " + Assert.assertEquals("ret:0, Added trace span receiver 1 with " +
"configuration dfs.htrace.local-file-span-receiver.path = " + tracePath + NEWLINE, "configuration dfs.htrace.local-file-span-receiver.path = " + tracePath + NEWLINE,
runTraceCommand(trace, "-add", "-host", getHostPortForNN(cluster), runTraceCommand(trace, "-add", "-host", getHostPortForNN(cluster),
"-class", "org.apache.htrace.impl.LocalFileSpanReceiver", "-class", "org.apache.htrace.core.LocalFileSpanReceiver",
"-Cdfs.htrace.local-file-span-receiver.path=" + tracePath)); "-Cdfs.htrace.local-file-span-receiver.path=" + tracePath));
String list = String list =
runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster)); runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster));
Assert.assertTrue(list.startsWith("ret:0")); Assert.assertTrue(list.startsWith("ret:0"));
Assert.assertTrue(list.contains("1 org.apache.htrace.impl.LocalFileSpanReceiver")); Assert.assertTrue(list.contains("1 org.apache.htrace.core.LocalFileSpanReceiver"));
Assert.assertEquals("ret:0, Removed trace span receiver 1" + NEWLINE, Assert.assertEquals("ret:0, Removed trace span receiver 1" + NEWLINE,
runTraceCommand(trace, "-remove", "1", "-host", runTraceCommand(trace, "-remove", "1", "-host",
getHostPortForNN(cluster))); getHostPortForNN(cluster)));

View File

@ -17,50 +17,72 @@
*/ */
package org.apache.hadoop.tracing; package org.apache.hadoop.tracing;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.htrace.Sampler; import org.apache.htrace.core.Sampler;
import org.apache.htrace.Span; import org.apache.htrace.core.Span;
import org.apache.htrace.Trace; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.TraceScope; import org.apache.htrace.core.Tracer;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
public class TestTracing { public class TestTracing {
private static Configuration conf;
private static MiniDFSCluster cluster; private static MiniDFSCluster cluster;
private static DistributedFileSystem dfs; private static DistributedFileSystem dfs;
private Tracer prevTracer;
private final static Configuration TRACING_CONF;
private final static Configuration NO_TRACING_CONF;
static {
NO_TRACING_CONF = new Configuration();
NO_TRACING_CONF.setLong("dfs.blocksize", 100 * 1024);
TRACING_CONF = new Configuration(NO_TRACING_CONF);
TRACING_CONF.set(CommonConfigurationKeys.FS_CLIENT_HTRACE_PREFIX +
Tracer.SPAN_RECEIVER_CLASSES_KEY,
SetSpanReceiver.class.getName());
TRACING_CONF.set(CommonConfigurationKeys.FS_CLIENT_HTRACE_PREFIX +
Tracer.SAMPLER_CLASSES_KEY, "AlwaysSampler");
}
@Test @Test
public void testTracing() throws Exception { public void testTracing() throws Exception {
// write and read without tracing started // write and read without tracing started
String fileName = "testTracingDisabled.dat"; String fileName = "testTracingDisabled.dat";
writeTestFile(fileName); writeTestFile(fileName);
Assert.assertTrue(SetSpanReceiver.size() == 0); Assert.assertEquals(0, SetSpanReceiver.size());
readTestFile(fileName); readTestFile(fileName);
Assert.assertTrue(SetSpanReceiver.size() == 0); Assert.assertEquals(0, SetSpanReceiver.size());
writeWithTracing(); writeTestFile("testReadTraceHooks.dat");
readWithTracing();
FsTracer.clear();
Tracer tracer = FsTracer.get(TRACING_CONF);
writeWithTracing(tracer);
readWithTracing(tracer);
} }
public void writeWithTracing() throws Exception { private void writeWithTracing(Tracer tracer) throws Exception {
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
TraceScope ts = Trace.startSpan("testWriteTraceHooks", Sampler.ALWAYS); TraceScope ts = tracer.newScope("testWriteTraceHooks");
writeTestFile("testWriteTraceHooks.dat"); writeTestFile("testWriteTraceHooks.dat");
long endTime = System.currentTimeMillis(); long endTime = System.currentTimeMillis();
ts.close(); ts.close();
@ -107,7 +129,8 @@ public class TestTracing {
}; };
for (String desc : spansInTopTrace) { for (String desc : spansInTopTrace) {
for (Span span : map.get(desc)) { for (Span span : map.get(desc)) {
Assert.assertEquals(ts.getSpan().getTraceId(), span.getTraceId()); Assert.assertEquals(ts.getSpan().getSpanId().getHigh(),
span.getSpanId().getHigh());
} }
} }
@ -120,12 +143,10 @@ public class TestTracing {
SetSpanReceiver.clear(); SetSpanReceiver.clear();
} }
public void readWithTracing() throws Exception { private void readWithTracing(Tracer tracer) throws Exception {
String fileName = "testReadTraceHooks.dat";
writeTestFile(fileName);
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
TraceScope ts = Trace.startSpan("testReadTraceHooks", Sampler.ALWAYS); TraceScope ts = tracer.newScope("testReadTraceHooks");
readTestFile(fileName); readTestFile("testReadTraceHooks.dat");
ts.close(); ts.close();
long endTime = System.currentTimeMillis(); long endTime = System.currentTimeMillis();
@ -150,7 +171,11 @@ public class TestTracing {
// There should only be one trace id as it should all be homed in the // There should only be one trace id as it should all be homed in the
// top trace. // top trace.
for (Span span : SetSpanReceiver.getSpans()) { for (Span span : SetSpanReceiver.getSpans()) {
Assert.assertEquals(ts.getSpan().getTraceId(), span.getTraceId()); System.out.println(span.toJson());
}
for (Span span : SetSpanReceiver.getSpans()) {
Assert.assertEquals(ts.getSpan().getSpanId().getHigh(),
span.getSpanId().getHigh());
} }
SetSpanReceiver.clear(); SetSpanReceiver.clear();
} }
@ -185,18 +210,9 @@ public class TestTracing {
} }
} }
@BeforeClass
public static void setup() throws IOException {
conf = new Configuration();
conf.setLong("dfs.blocksize", 100 * 1024);
conf.set(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX +
SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX,
SetSpanReceiver.class.getName());
}
@Before @Before
public void startCluster() throws IOException { public void startCluster() throws IOException {
cluster = new MiniDFSCluster.Builder(conf) cluster = new MiniDFSCluster.Builder(NO_TRACING_CONF)
.numDataNodes(3) .numDataNodes(3)
.build(); .build();
cluster.waitActive(); cluster.waitActive();
@ -207,6 +223,6 @@ public class TestTracing {
@After @After
public void shutDown() throws IOException { public void shutDown() throws IOException {
cluster.shutdown(); cluster.shutdown();
FsTracer.clear();
} }
} }

View File

@ -23,6 +23,8 @@ import java.io.IOException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
@ -33,9 +35,8 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory; import org.apache.hadoop.net.unix.TemporarySocketDirectory;
import org.apache.hadoop.util.NativeCodeLoader; import org.apache.hadoop.util.NativeCodeLoader;
import org.apache.htrace.Sampler; import org.apache.htrace.core.TraceScope;
import org.apache.htrace.Trace; import org.apache.htrace.core.Tracer;
import org.apache.htrace.TraceScope;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -44,7 +45,6 @@ public class TestTracingShortCircuitLocalRead {
private static Configuration conf; private static Configuration conf;
private static MiniDFSCluster cluster; private static MiniDFSCluster cluster;
private static DistributedFileSystem dfs; private static DistributedFileSystem dfs;
private static SpanReceiverHost spanReceiverHost;
private static TemporarySocketDirectory sockDir; private static TemporarySocketDirectory sockDir;
static final Path TEST_PATH = new Path("testShortCircuitTraceHooks"); static final Path TEST_PATH = new Path("testShortCircuitTraceHooks");
static final int TEST_LENGTH = 1234; static final int TEST_LENGTH = 1234;
@ -64,9 +64,12 @@ public class TestTracingShortCircuitLocalRead {
public void testShortCircuitTraceHooks() throws IOException { public void testShortCircuitTraceHooks() throws IOException {
assumeTrue(NativeCodeLoader.isNativeCodeLoaded() && !Path.WINDOWS); assumeTrue(NativeCodeLoader.isNativeCodeLoaded() && !Path.WINDOWS);
conf = new Configuration(); conf = new Configuration();
conf.set(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX + conf.set(TraceUtils.DEFAULT_HADOOP_PREFIX +
SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX, Tracer.SPAN_RECEIVER_CLASSES_KEY,
SetSpanReceiver.class.getName()); SetSpanReceiver.class.getName());
conf.set(TraceUtils.DEFAULT_HADOOP_PREFIX +
Tracer.SAMPLER_CLASSES_KEY,
"AlwaysSampler");
conf.setLong("dfs.blocksize", 100 * 1024); conf.setLong("dfs.blocksize", 100 * 1024);
conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true); conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false); conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false);
@ -81,7 +84,8 @@ public class TestTracingShortCircuitLocalRead {
try { try {
DFSTestUtil.createFile(dfs, TEST_PATH, TEST_LENGTH, (short)1, 5678L); DFSTestUtil.createFile(dfs, TEST_PATH, TEST_LENGTH, (short)1, 5678L);
TraceScope ts = Trace.startSpan("testShortCircuitTraceHooks", Sampler.ALWAYS); TraceScope ts = FsTracer.get(conf).
newScope("testShortCircuitTraceHooks");
FSDataInputStream stream = dfs.open(TEST_PATH); FSDataInputStream stream = dfs.open(TEST_PATH);
byte buf[] = new byte[TEST_LENGTH]; byte buf[] = new byte[TEST_LENGTH];
IOUtils.readFully(stream, buf, 0, TEST_LENGTH); IOUtils.readFully(stream, buf, 0, TEST_LENGTH);

View File

@ -806,8 +806,8 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.htrace</groupId> <groupId>org.apache.htrace</groupId>
<artifactId>htrace-core</artifactId> <artifactId>htrace-core4</artifactId>
<version>3.2.0-incubating</version> <version>4.0.1-incubating</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.jdom</groupId> <groupId>org.jdom</groupId>