HDFS-7189. Add trace spans for DFSClient metadata operations. (Colin P. McCabe via yliu)

This commit is contained in:
yliu 2015-01-16 00:21:38 +08:00
parent 2e4df87104
commit bdbf13ac46
8 changed files with 476 additions and 138 deletions

View File

@ -247,6 +247,9 @@ Release 2.7.0 - UNRELEASED
HDFS-7457. DatanodeID generates excessive garbage. (daryn via kihwal) HDFS-7457. DatanodeID generates excessive garbage. (daryn via kihwal)
HDFS-7189. Add trace spans for DFSClient metadata operations. (Colin P.
McCabe via yliu)
OPTIMIZATIONS OPTIMIZATIONS
HDFS-7454. Reduce memory footprint for AclEntries in NameNode. HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

View File

@ -51,6 +51,10 @@ import org.apache.hadoop.security.token.Token;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import org.htrace.Sampler;
import org.htrace.Span;
import org.htrace.Trace;
import org.htrace.TraceScope;
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Unstable @InterfaceStability.Unstable
@ -71,7 +75,7 @@ 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) { int timeout, boolean connectToDnViaHostname, Span parent) {
if (datanodeBlocks.isEmpty()) { if (datanodeBlocks.isEmpty()) {
return Lists.newArrayList(); return Lists.newArrayList();
@ -111,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); connectToDnViaHostname, parent);
callables.add(callable); callables.add(callable);
} }
return callables; return callables;
@ -135,7 +139,7 @@ class BlockStorageLocationUtil {
List<VolumeBlockLocationCallable> callables = List<VolumeBlockLocationCallable> callables =
createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs, createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs,
connectToDnViaHostname); connectToDnViaHostname, Trace.currentSpan());
// 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 =
@ -319,11 +323,12 @@ 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;
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) { boolean connectToDnViaHostname, Span parentSpan) {
this.configuration = configuration; this.configuration = configuration;
this.timeout = timeout; this.timeout = timeout;
this.datanode = datanode; this.datanode = datanode;
@ -331,6 +336,7 @@ class BlockStorageLocationUtil {
this.blockIds = blockIds; this.blockIds = blockIds;
this.dnTokens = dnTokens; this.dnTokens = dnTokens;
this.connectToDnViaHostname = connectToDnViaHostname; this.connectToDnViaHostname = connectToDnViaHostname;
this.parentSpan = parentSpan;
} }
public DatanodeInfo getDatanodeInfo() { public DatanodeInfo getDatanodeInfo() {
@ -342,6 +348,8 @@ class BlockStorageLocationUtil {
HdfsBlocksMetadata metadata = null; HdfsBlocksMetadata metadata = null;
// Create the RPC proxy and make the RPC // Create the RPC proxy and make the RPC
ClientDatanodeProtocol cdp = null; ClientDatanodeProtocol cdp = null;
TraceScope scope =
Trace.startSpan("getHdfsBlocksMetadata", parentSpan);
try { try {
cdp = DFSUtil.createClientDatanodeProtocolProxy(datanode, configuration, cdp = DFSUtil.createClientDatanodeProtocolProxy(datanode, configuration,
timeout, connectToDnViaHostname); timeout, connectToDnViaHostname);
@ -350,6 +358,7 @@ class BlockStorageLocationUtil {
// Bubble this up to the caller, handle with the Future // Bubble this up to the caller, handle with the Future
throw e; throw e;
} finally { } finally {
scope.close();
if (cdp != null) { if (cdp != null) {
RPC.stopProxy(cdp); RPC.stopProxy(cdp);
} }

View File

@ -26,6 +26,9 @@ 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.htrace.Sampler;
import org.htrace.Trace;
import org.htrace.TraceScope;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -44,6 +47,11 @@ public class DFSInotifyEventInputStream {
public static Logger LOG = LoggerFactory.getLogger(DFSInotifyEventInputStream public static Logger LOG = LoggerFactory.getLogger(DFSInotifyEventInputStream
.class); .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;
@ -59,12 +67,15 @@ public class DFSInotifyEventInputStream {
private static final int INITIAL_WAIT_MS = 10; private static final int INITIAL_WAIT_MS = 10;
DFSInotifyEventInputStream(ClientProtocol namenode) throws IOException { DFSInotifyEventInputStream(Sampler<?> traceSampler, ClientProtocol namenode)
this(namenode, namenode.getCurrentEditLogTxid()); // only consider new txn's throws IOException {
// Only consider new transaction IDs.
this(traceSampler, namenode, namenode.getCurrentEditLogTxid());
} }
DFSInotifyEventInputStream(ClientProtocol namenode, long lastReadTxid) DFSInotifyEventInputStream(Sampler traceSampler, ClientProtocol namenode,
throws IOException { 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;
@ -87,6 +98,9 @@ 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 =
Trace.startSpan("inotifyPoll", traceSampler);
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) {
LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN"); LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
@ -121,6 +135,9 @@ public class DFSInotifyEventInputStream {
} else { } else {
return null; return null;
} }
} finally {
scope.close();
}
} }
/** /**
@ -163,10 +180,12 @@ 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);
EventBatch next = null;
try {
long initialTime = Time.monotonicNow(); long initialTime = Time.monotonicNow();
long totalWait = TimeUnit.MILLISECONDS.convert(time, tu); long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
long nextWait = INITIAL_WAIT_MS; long nextWait = INITIAL_WAIT_MS;
EventBatch next = null;
while ((next = poll()) == null) { while ((next = poll()) == null) {
long timeLeft = totalWait - (Time.monotonicNow() - initialTime); long timeLeft = totalWait - (Time.monotonicNow() - initialTime);
if (timeLeft <= 0) { if (timeLeft <= 0) {
@ -181,7 +200,9 @@ public class DFSInotifyEventInputStream {
nextWait); nextWait);
Thread.sleep(nextWait); Thread.sleep(nextWait);
} }
} finally {
scope.close();
}
return next; return next;
} }
@ -196,7 +217,9 @@ public class DFSInotifyEventInputStream {
*/ */
public EventBatch take() throws IOException, InterruptedException, public EventBatch take() throws IOException, InterruptedException,
MissingEventsException { MissingEventsException {
TraceScope scope = Trace.startSpan("inotifyTake", traceSampler);
EventBatch next = null; EventBatch next = null;
try {
int nextWaitMin = INITIAL_WAIT_MS; int nextWaitMin = INITIAL_WAIT_MS;
while ((next = poll()) == null) { while ((next = poll()) == null) {
// sleep for a random period between nextWaitMin and nextWaitMin * 2 // sleep for a random period between nextWaitMin and nextWaitMin * 2
@ -207,6 +230,9 @@ public class DFSInotifyEventInputStream {
// the maximum sleep is 2 minutes // the maximum sleep is 2 minutes
nextWaitMin = Math.min(60000, nextWaitMin * 2); nextWaitMin = Math.min(60000, nextWaitMin * 2);
} }
} finally {
scope.close();
}
return next; return next;
} }

View File

@ -27,6 +27,9 @@ import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.htrace.Sampler;
import org.htrace.Trace;
import org.htrace.TraceScope;
/** /**
* CacheDirectiveIterator is a remote iterator that iterates cache directives. * CacheDirectiveIterator is a remote iterator that iterates cache directives.
@ -39,12 +42,14 @@ public class CacheDirectiveIterator
private CacheDirectiveInfo filter; private CacheDirectiveInfo filter;
private final ClientProtocol namenode; private final ClientProtocol namenode;
private final Sampler<?> traceSampler;
public CacheDirectiveIterator(ClientProtocol namenode, public CacheDirectiveIterator(ClientProtocol namenode,
CacheDirectiveInfo filter) { CacheDirectiveInfo filter, Sampler<?> traceSampler) {
super(0L); super(0L);
this.namenode = namenode; this.namenode = namenode;
this.filter = filter; this.filter = filter;
this.traceSampler = traceSampler;
} }
private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) { private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) {
@ -89,6 +94,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);
try { try {
entries = namenode.listCacheDirectives(prevKey, filter); entries = namenode.listCacheDirectives(prevKey, filter);
} catch (IOException e) { } catch (IOException e) {
@ -110,6 +116,8 @@ public class CacheDirectiveIterator
"Did not find requested id " + id); "Did not find requested id " + id);
} }
throw e; throw e;
} finally {
scope.close();
} }
Preconditions.checkNotNull(entries); Preconditions.checkNotNull(entries);
return entries; return entries;

View File

@ -23,6 +23,9 @@ 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.htrace.Sampler;
import org.htrace.Trace;
import org.htrace.TraceScope;
/** /**
* CachePoolIterator is a remote iterator that iterates cache pools. * CachePoolIterator is a remote iterator that iterates cache pools.
@ -34,16 +37,23 @@ public class CachePoolIterator
extends BatchedRemoteIterator<String, CachePoolEntry> { extends BatchedRemoteIterator<String, CachePoolEntry> {
private final ClientProtocol namenode; private final ClientProtocol namenode;
private final Sampler traceSampler;
public CachePoolIterator(ClientProtocol namenode) { public CachePoolIterator(ClientProtocol namenode, Sampler traceSampler) {
super(""); super("");
this.namenode = namenode; this.namenode = namenode;
this.traceSampler = traceSampler;
} }
@Override @Override
public BatchedEntries<CachePoolEntry> makeRequest(String prevKey) public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
throws IOException { throws IOException {
TraceScope scope = Trace.startSpan("listCachePools", traceSampler);
try {
return namenode.listCachePools(prevKey); return namenode.listCachePools(prevKey);
} finally {
scope.close();
}
} }
@Override @Override

View File

@ -23,6 +23,9 @@ 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.htrace.Sampler;
import org.htrace.Trace;
import org.htrace.TraceScope;
/** /**
* EncryptionZoneIterator is a remote iterator that iterates over encryption * EncryptionZoneIterator is a remote iterator that iterates over encryption
@ -34,16 +37,24 @@ public class EncryptionZoneIterator
extends BatchedRemoteIterator<Long, EncryptionZone> { extends BatchedRemoteIterator<Long, EncryptionZone> {
private final ClientProtocol namenode; private final ClientProtocol namenode;
private final Sampler<?> traceSampler;
public EncryptionZoneIterator(ClientProtocol namenode) { public EncryptionZoneIterator(ClientProtocol namenode,
Sampler<?> traceSampler) {
super(Long.valueOf(0)); super(Long.valueOf(0));
this.namenode = namenode; this.namenode = namenode;
this.traceSampler = traceSampler;
} }
@Override @Override
public BatchedEntries<EncryptionZone> makeRequest(Long prevId) public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
throws IOException { throws IOException {
TraceScope scope = Trace.startSpan("listEncryptionZones", traceSampler);
try {
return namenode.listEncryptionZones(prevId); return namenode.listEncryptionZones(prevId);
} finally {
scope.close();
}
} }
@Override @Override

View File

@ -88,6 +88,7 @@ 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.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;
@ -908,7 +909,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); new CacheDirectiveIterator(nnRpc, null, Sampler.NEVER);
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());