HDFS-13702. Remove HTrace hooks from DFSClient to reduce CPU usage. Contributed by Todd Lipcon.

This commit is contained in:
Andrew Wang 2018-07-02 12:11:06 +02:00
parent 6ba9974108
commit 5d748bd056
12 changed files with 70 additions and 180 deletions

View File

@ -3070,25 +3070,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
return scope;
}
/**
* Full detailed tracing for read requests: path, position in the file,
* and length.
*
* @param reqLen requested length
*/
TraceScope newReaderTraceScope(String description, String path, long pos,
int reqLen) {
TraceScope scope = newPathTraceScope(description, path);
scope.addKVAnnotation("pos", Long.toString(pos));
scope.addKVAnnotation("reqLen", Integer.toString(reqLen));
return scope;
}
/** Add the returned length info to the scope. */
void addRetLenToReaderScope(TraceScope scope, int retLen) {
scope.addKVAnnotation("retLen", Integer.toString(retLen));
}
/**
* Get the erasure coding policy information for the specified path
*

View File

@ -85,8 +85,6 @@ import org.apache.hadoop.util.IdentityHashStore;
import org.apache.hadoop.util.StopWatch;
import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope;
import org.apache.htrace.core.Tracer;
import com.google.common.annotations.VisibleForTesting;
@ -641,7 +639,6 @@ public class DFSInputStream extends FSInputStream
setClientCacheContext(dfsClient.getClientContext()).
setUserGroupInformation(dfsClient.ugi).
setConfiguration(dfsClient.getConfiguration()).
setTracer(dfsClient.getTracer()).
build();
}
@ -821,31 +818,14 @@ public class DFSInputStream extends FSInputStream
}
ReaderStrategy byteArrayReader =
new ByteArrayStrategy(buf, off, len, readStatistics, dfsClient);
try (TraceScope scope =
dfsClient.newReaderTraceScope("DFSInputStream#byteArrayRead",
src, getPos(), len)) {
int retLen = readWithStrategy(byteArrayReader);
if (retLen < len) {
dfsClient.addRetLenToReaderScope(scope, retLen);
}
return retLen;
}
return readWithStrategy(byteArrayReader);
}
@Override
public synchronized int read(final ByteBuffer buf) throws IOException {
ReaderStrategy byteBufferReader =
new ByteBufferStrategy(buf, readStatistics, dfsClient);
int reqLen = buf.remaining();
try (TraceScope scope =
dfsClient.newReaderTraceScope("DFSInputStream#byteBufferRead",
src, getPos(), reqLen)){
int retLen = readWithStrategy(byteBufferReader);
if (retLen < reqLen) {
dfsClient.addRetLenToReaderScope(scope, retLen);
}
return retLen;
}
return readWithStrategy(byteBufferReader);
}
private DNAddrPair chooseDataNode(LocatedBlock block,
@ -1026,16 +1006,12 @@ public class DFSInputStream extends FSInputStream
final ByteBuffer bb,
final CorruptedBlocks corruptedBlocks,
final int hedgedReadId) {
final SpanId parentSpanId = Tracer.getCurrentSpanId();
return new Callable<ByteBuffer>() {
@Override
public ByteBuffer call() throws Exception {
DFSClientFaultInjector.get().sleepBeforeHedgedGet();
try (TraceScope ignored = dfsClient.getTracer().
newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks);
return bb;
}
actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks);
return bb;
}
};
}
@ -1336,16 +1312,8 @@ public class DFSInputStream extends FSInputStream
if (length == 0) {
return 0;
}
try (TraceScope scope = dfsClient.
newReaderTraceScope("DFSInputStream#byteArrayPread",
src, position, length)) {
ByteBuffer bb = ByteBuffer.wrap(buffer, offset, length);
int retLen = pread(position, bb);
if (retLen < length) {
dfsClient.addRetLenToReaderScope(scope, retLen);
}
return retLen;
}
ByteBuffer bb = ByteBuffer.wrap(buffer, offset, length);
return pread(position, bb);
}
private int pread(long position, ByteBuffer buffer)

View File

@ -75,7 +75,6 @@ import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -189,11 +188,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
*/
private Configuration configuration;
/**
* The HTrace tracer to use.
*/
private Tracer tracer;
/**
* 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.
@ -298,11 +292,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return this;
}
public BlockReaderFactory setTracer(Tracer tracer) {
this.tracer = tracer;
return this;
}
@VisibleForTesting
public static void setFailureInjectorForTesting(FailureInjector injector) {
failureInjector = injector;
@ -451,7 +440,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
try {
return BlockReaderLocalLegacy.newBlockReader(conf,
userGroupInformation, configuration, fileName, block, token,
datanode, startOffset, length, storageType, tracer);
datanode, startOffset, length, storageType);
} catch (RemoteException remoteException) {
ioe = remoteException.unwrapRemoteException(
InvalidToken.class, AccessControlException.class);
@ -509,7 +498,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
setVerifyChecksum(verifyChecksum).
setCachingStrategy(cachingStrategy).
setStorageType(storageType).
setTracer(tracer).
build();
}
@ -860,7 +848,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return BlockReaderRemote.newBlockReader(
fileName, block, token, startOffset, length,
verifyChecksum, clientName, peer, datanode,
clientContext.getPeerCache(), cachingStrategy, tracer,
clientContext.getPeerCache(), cachingStrategy,
networkDistance);
}

View File

@ -35,8 +35,6 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DirectBufferPool;
import org.apache.hadoop.util.Timer;
import org.apache.htrace.core.TraceScope;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -83,7 +81,6 @@ class BlockReaderLocal implements BlockReader {
private long dataPos;
private ExtendedBlock block;
private StorageType storageType;
private Tracer tracer;
private ShortCircuitConf shortCircuitConf;
public Builder(ShortCircuitConf conf) {
@ -131,11 +128,6 @@ class BlockReaderLocal implements BlockReader {
return this;
}
public Builder setTracer(Tracer tracer) {
this.tracer = tracer;
return this;
}
public BlockReaderLocal build() {
Preconditions.checkNotNull(replica);
return new BlockReaderLocal(this);
@ -244,11 +236,6 @@ class BlockReaderLocal implements BlockReader {
*/
private StorageType storageType;
/**
* The Tracer to use.
*/
private final Tracer tracer;
private BlockReaderLocal(Builder builder) {
this.replica = builder.replica;
this.dataIn = replica.getDataStream().getChannel();
@ -278,7 +265,6 @@ class BlockReaderLocal implements BlockReader {
}
this.maxReadaheadLength = maxReadaheadChunks * bytesPerChecksum;
this.storageType = builder.storageType;
this.tracer = builder.tracer;
if (builder.shortCircuitConf.isScrMetricsEnabled()) {
metricsInitializationLock.lock();
@ -360,52 +346,49 @@ class BlockReaderLocal implements BlockReader {
*/
private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
throws IOException {
try (TraceScope ignored = tracer.newScope(
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) {
int total = 0;
long startDataPos = dataPos;
int startBufPos = buf.position();
while (buf.hasRemaining()) {
int nRead = blockReaderIoProvider.read(dataIn, buf, dataPos);
if (nRead < 0) {
break;
}
dataPos += nRead;
total += nRead;
}
if (canSkipChecksum) {
freeChecksumBufIfExists();
return total;
}
if (total > 0) {
try {
buf.limit(buf.position());
buf.position(startBufPos);
createChecksumBufIfNeeded();
int checksumsNeeded = (total + bytesPerChecksum - 1) /
bytesPerChecksum;
checksumBuf.clear();
checksumBuf.limit(checksumsNeeded * checksumSize);
long checksumPos = BlockMetadataHeader.getHeaderSize()
+ ((startDataPos / bytesPerChecksum) * checksumSize);
while (checksumBuf.hasRemaining()) {
int nRead = checksumIn.read(checksumBuf, checksumPos);
if (nRead < 0) {
throw new IOException("Got unexpected checksum file EOF at " +
checksumPos + ", block file position " + startDataPos +
" for block " + block + " of file " + filename);
}
checksumPos += nRead;
}
checksumBuf.flip();
checksum.verifyChunkedSums(buf, checksumBuf, filename, startDataPos);
} finally {
buf.position(buf.limit());
}
int total = 0;
long startDataPos = dataPos;
int startBufPos = buf.position();
while (buf.hasRemaining()) {
int nRead = blockReaderIoProvider.read(dataIn, buf, dataPos);
if (nRead < 0) {
break;
}
dataPos += nRead;
total += nRead;
}
if (canSkipChecksum) {
freeChecksumBufIfExists();
return total;
}
if (total > 0) {
try {
buf.limit(buf.position());
buf.position(startBufPos);
createChecksumBufIfNeeded();
int checksumsNeeded = (total + bytesPerChecksum - 1) /
bytesPerChecksum;
checksumBuf.clear();
checksumBuf.limit(checksumsNeeded * checksumSize);
long checksumPos = BlockMetadataHeader.getHeaderSize()
+ ((startDataPos / bytesPerChecksum) * checksumSize);
while (checksumBuf.hasRemaining()) {
int nRead = checksumIn.read(checksumBuf, checksumPos);
if (nRead < 0) {
throw new IOException("Got unexpected checksum file EOF at " +
checksumPos + ", block file position " + startDataPos +
" for block " + block + " of file " + filename);
}
checksumPos += nRead;
}
checksumBuf.flip();
checksum.verifyChunkedSums(buf, checksumBuf, filename, startDataPos);
} finally {
buf.position(buf.limit());
}
}
return total;
}
private boolean createNoChecksumContext() {

View File

@ -51,8 +51,6 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DirectBufferPool;
import org.apache.htrace.core.TraceScope;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -184,7 +182,6 @@ class BlockReaderLocalLegacy implements BlockReader {
private long startOffset;
private final String filename;
private long blockId;
private final Tracer tracer;
/**
* The only way this object can be instantiated.
@ -193,8 +190,8 @@ class BlockReaderLocalLegacy implements BlockReader {
UserGroupInformation userGroupInformation,
Configuration configuration, String file, ExtendedBlock blk,
Token<BlockTokenIdentifier> token, DatanodeInfo node,
long startOffset, long length, StorageType storageType,
Tracer tracer) throws IOException {
long startOffset, long length, StorageType storageType)
throws IOException {
final ShortCircuitConf scConf = conf.getShortCircuitConf();
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
.getIpcPort());
@ -239,11 +236,10 @@ class BlockReaderLocalLegacy implements BlockReader {
long firstChunkOffset = startOffset
- (startOffset % checksum.getBytesPerChecksum());
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn,
tracer);
startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn);
} else {
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
startOffset, dataIn, tracer);
startOffset, dataIn);
}
} catch (IOException e) {
// remove from cache
@ -320,17 +316,17 @@ class BlockReaderLocalLegacy implements BlockReader {
}
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
ExtendedBlock block, long startOffset, FileInputStream dataIn,
Tracer tracer) throws IOException {
ExtendedBlock block, long startOffset, FileInputStream dataIn)
throws IOException {
this(conf, hdfsfile, block, startOffset,
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
dataIn, startOffset, null, tracer);
dataIn, startOffset, null);
}
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
ExtendedBlock block, long startOffset, DataChecksum checksum,
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
FileInputStream checksumIn, Tracer tracer) throws IOException {
FileInputStream checksumIn) throws IOException {
this.filename = hdfsfile;
this.checksum = checksum;
this.verifyChecksum = verifyChecksum;
@ -369,7 +365,6 @@ class BlockReaderLocalLegacy implements BlockReader {
bufferPool.returnBuffer(checksumBuff);
}
}
this.tracer = tracer;
}
/**
@ -377,23 +372,20 @@ class BlockReaderLocalLegacy implements BlockReader {
*/
private int fillBuffer(FileInputStream stream, ByteBuffer buf)
throws IOException {
try (TraceScope ignored = tracer.
newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")")) {
int bytesRead = stream.getChannel().read(buf);
if (bytesRead < 0) {
int bytesRead = stream.getChannel().read(buf);
if (bytesRead < 0) {
//EOF
return bytesRead;
}
while (buf.remaining() > 0) {
int n = stream.getChannel().read(buf);
if (n < 0) {
//EOF
return bytesRead;
}
while (buf.remaining() > 0) {
int n = stream.getChannel().read(buf);
if (n < 0) {
//EOF
return bytesRead;
}
bytesRead += n;
}
return bytesRead;
bytesRead += n;
}
return bytesRead;
}
/**

View File

@ -49,11 +49,9 @@ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum;
import org.apache.htrace.core.TraceScope;
import com.google.common.annotations.VisibleForTesting;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -121,8 +119,6 @@ public class BlockReaderRemote implements BlockReader {
private boolean sentStatusCode = false;
private final Tracer tracer;
private final int networkDistance;
@VisibleForTesting
@ -139,10 +135,7 @@ public class BlockReaderRemote implements BlockReader {
if (curDataSlice == null ||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
try (TraceScope ignored = tracer.newScope(
"BlockReaderRemote2#readNextPacket(" + blockId + ")")) {
readNextPacket();
}
readNextPacket();
}
LOG.trace("Finishing read #{}", randomId);
@ -163,10 +156,7 @@ public class BlockReaderRemote implements BlockReader {
public synchronized int read(ByteBuffer buf) throws IOException {
if (curDataSlice == null ||
(curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) {
try (TraceScope ignored = tracer.newScope(
"BlockReaderRemote2#readNextPacket(" + blockId + ")")) {
readNextPacket();
}
readNextPacket();
}
if (curDataSlice.remaining() == 0) {
// we're at EOF now
@ -280,7 +270,6 @@ public class BlockReaderRemote implements BlockReader {
long startOffset, long firstChunkOffset,
long bytesToRead, Peer peer,
DatanodeID datanodeID, PeerCache peerCache,
Tracer tracer,
int networkDistance) {
// Path is used only for printing block and file information in debug
this.peer = peer;
@ -300,7 +289,6 @@ public class BlockReaderRemote implements BlockReader {
this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
bytesPerChecksum = this.checksum.getBytesPerChecksum();
checksumSize = this.checksum.getChecksumSize();
this.tracer = tracer;
this.networkDistance = networkDistance;
}
@ -397,7 +385,6 @@ public class BlockReaderRemote implements BlockReader {
Peer peer, DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer,
int networkDistance) throws IOException {
// in and out will be closed when sock is closed (by the caller)
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
@ -431,7 +418,7 @@ public class BlockReaderRemote implements BlockReader {
return new BlockReaderRemote(file, block.getBlockId(), checksum,
verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
peerCache, tracer, networkDistance);
peerCache, networkDistance);
}
static void checkSuccess(

View File

@ -128,7 +128,7 @@ class StripedBlockReader {
return BlockReaderRemote.newBlockReader(
"dummy", block, blockToken, offsetInBlock,
block.getNumBytes() - offsetInBlock, true, "", peer, source,
null, stripedReader.getCachingStrategy(), datanode.getTracer(), -1);
null, stripedReader.getCachingStrategy(), -1);
} catch (IOException e) {
LOG.info("Exception while creating remote block reader, datanode {}",
source, e);

View File

@ -1032,7 +1032,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
setCachingStrategy(CachingStrategy.newDropBehind()).
setClientCacheContext(dfs.getClientContext()).
setConfiguration(namenode.getConf()).
setTracer(tracer).
setRemotePeerFactory(new RemotePeerFactory() {
@Override
public Peer newConnectedPeer(InetSocketAddress addr,

View File

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

View File

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

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
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.hdfs.BlockReader;
import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
@ -167,7 +166,6 @@ public class TestBlockTokenWithDFS {
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
setClientCacheContext(ClientContext.getFromConf(conf)).
setConfiguration(conf).
setTracer(FsTracer.get(conf)).
setRemotePeerFactory(new RemotePeerFactory() {
@Override
public Peer newConnectedPeer(InetSocketAddress addr,

View File

@ -40,7 +40,6 @@ import org.apache.commons.io.filefilter.TrueFileFilter;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.ClientContext;
@ -655,7 +654,6 @@ public class TestDataNodeVolumeFailure {
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
setClientCacheContext(ClientContext.getFromConf(conf)).
setConfiguration(conf).
setTracer(FsTracer.get(conf)).
setRemotePeerFactory(new RemotePeerFactory() {
@Override
public Peer newConnectedPeer(InetSocketAddress addr,