|
@@ -71,6 +71,7 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
|
|
|
import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
|
|
|
+import org.apache.hadoop.hdfs.util.IOUtilsClient;
|
|
|
import org.apache.hadoop.io.ByteBufferPool;
|
|
|
import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
@@ -145,94 +146,6 @@ public class DFSInputStream extends FSInputStream
|
|
|
return extendedReadBuffers;
|
|
|
}
|
|
|
|
|
|
- public static class ReadStatistics {
|
|
|
- public ReadStatistics() {
|
|
|
- clear();
|
|
|
- }
|
|
|
-
|
|
|
- public ReadStatistics(ReadStatistics rhs) {
|
|
|
- this.totalBytesRead = rhs.getTotalBytesRead();
|
|
|
- this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
|
|
|
- this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
|
|
|
- this.totalZeroCopyBytesRead = rhs.getTotalZeroCopyBytesRead();
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return The total bytes read. This will always be at least as
|
|
|
- * high as the other numbers, since it includes all of them.
|
|
|
- */
|
|
|
- public long getTotalBytesRead() {
|
|
|
- return totalBytesRead;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return The total local bytes read. This will always be at least
|
|
|
- * as high as totalShortCircuitBytesRead, since all short-circuit
|
|
|
- * reads are also local.
|
|
|
- */
|
|
|
- public long getTotalLocalBytesRead() {
|
|
|
- return totalLocalBytesRead;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return The total short-circuit local bytes read.
|
|
|
- */
|
|
|
- public long getTotalShortCircuitBytesRead() {
|
|
|
- return totalShortCircuitBytesRead;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return The total number of zero-copy bytes read.
|
|
|
- */
|
|
|
- public long getTotalZeroCopyBytesRead() {
|
|
|
- return totalZeroCopyBytesRead;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return The total number of bytes read which were not local.
|
|
|
- */
|
|
|
- public long getRemoteBytesRead() {
|
|
|
- return totalBytesRead - totalLocalBytesRead;
|
|
|
- }
|
|
|
-
|
|
|
- void addRemoteBytes(long amt) {
|
|
|
- this.totalBytesRead += amt;
|
|
|
- }
|
|
|
-
|
|
|
- void addLocalBytes(long amt) {
|
|
|
- this.totalBytesRead += amt;
|
|
|
- this.totalLocalBytesRead += amt;
|
|
|
- }
|
|
|
-
|
|
|
- void addShortCircuitBytes(long amt) {
|
|
|
- this.totalBytesRead += amt;
|
|
|
- this.totalLocalBytesRead += amt;
|
|
|
- this.totalShortCircuitBytesRead += amt;
|
|
|
- }
|
|
|
-
|
|
|
- void addZeroCopyBytes(long amt) {
|
|
|
- this.totalBytesRead += amt;
|
|
|
- this.totalLocalBytesRead += amt;
|
|
|
- this.totalShortCircuitBytesRead += amt;
|
|
|
- this.totalZeroCopyBytesRead += amt;
|
|
|
- }
|
|
|
-
|
|
|
- void clear() {
|
|
|
- this.totalBytesRead = 0;
|
|
|
- this.totalLocalBytesRead = 0;
|
|
|
- this.totalShortCircuitBytesRead = 0;
|
|
|
- this.totalZeroCopyBytesRead = 0;
|
|
|
- }
|
|
|
-
|
|
|
- private long totalBytesRead;
|
|
|
-
|
|
|
- private long totalLocalBytesRead;
|
|
|
-
|
|
|
- private long totalShortCircuitBytesRead;
|
|
|
-
|
|
|
- private long totalZeroCopyBytesRead;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* This variable tracks the number of failures since the start of the
|
|
|
* most recent user-facing operation. That is to say, it should be reset
|
|
@@ -767,116 +680,11 @@ public class DFSInputStream extends FSInputStream
|
|
|
return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Wraps different possible read implementations so that readBuffer can be
|
|
|
- * strategy-agnostic.
|
|
|
- */
|
|
|
- interface ReaderStrategy {
|
|
|
- int doRead(BlockReader blockReader, int off, int len)
|
|
|
- throws IOException;
|
|
|
-
|
|
|
- /**
|
|
|
- * Copy data from the src ByteBuffer into the read buffer.
|
|
|
- * @param src The src buffer where the data is copied from
|
|
|
- * @param offset Useful only when the ReadStrategy is based on a byte array.
|
|
|
- * Indicate the offset of the byte array for copy.
|
|
|
- * @param length Useful only when the ReadStrategy is based on a byte array.
|
|
|
- * Indicate the length of the data to copy.
|
|
|
- */
|
|
|
- int copyFrom(ByteBuffer src, int offset, int length);
|
|
|
- }
|
|
|
-
|
|
|
- protected void updateReadStatistics(ReadStatistics readStatistics,
|
|
|
- int nRead, BlockReader blockReader) {
|
|
|
- if (nRead <= 0) return;
|
|
|
- synchronized(infoLock) {
|
|
|
- if (blockReader.isShortCircuit()) {
|
|
|
- readStatistics.addShortCircuitBytes(nRead);
|
|
|
- } else if (blockReader.getNetworkDistance() == 0) {
|
|
|
- readStatistics.addLocalBytes(nRead);
|
|
|
- } else {
|
|
|
- readStatistics.addRemoteBytes(nRead);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Used to read bytes into a byte[]
|
|
|
- */
|
|
|
- private class ByteArrayStrategy implements ReaderStrategy {
|
|
|
- final byte[] buf;
|
|
|
-
|
|
|
- public ByteArrayStrategy(byte[] buf) {
|
|
|
- this.buf = buf;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public int doRead(BlockReader blockReader, int off, int len)
|
|
|
- throws IOException {
|
|
|
- int nRead = blockReader.read(buf, off, len);
|
|
|
- updateReadStatistics(readStatistics, nRead, blockReader);
|
|
|
- dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(),
|
|
|
- nRead);
|
|
|
- return nRead;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public int copyFrom(ByteBuffer src, int offset, int length) {
|
|
|
- ByteBuffer writeSlice = src.duplicate();
|
|
|
- writeSlice.get(buf, offset, length);
|
|
|
- return length;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Used to read bytes into a user-supplied ByteBuffer
|
|
|
- */
|
|
|
- protected class ByteBufferStrategy implements ReaderStrategy {
|
|
|
- final ByteBuffer buf;
|
|
|
- ByteBufferStrategy(ByteBuffer buf) {
|
|
|
- this.buf = buf;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public int doRead(BlockReader blockReader, int off, int len)
|
|
|
- throws IOException {
|
|
|
- int oldpos = buf.position();
|
|
|
- int oldlimit = buf.limit();
|
|
|
- boolean success = false;
|
|
|
- try {
|
|
|
- int ret = blockReader.read(buf);
|
|
|
- success = true;
|
|
|
- updateReadStatistics(readStatistics, ret, blockReader);
|
|
|
- dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(),
|
|
|
- ret);
|
|
|
- if (ret == 0) {
|
|
|
- DFSClient.LOG.warn("zero");
|
|
|
- }
|
|
|
- return ret;
|
|
|
- } finally {
|
|
|
- if (!success) {
|
|
|
- // Reset to original state so that retries work correctly.
|
|
|
- buf.position(oldpos);
|
|
|
- buf.limit(oldlimit);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public int copyFrom(ByteBuffer src, int offset, int length) {
|
|
|
- ByteBuffer writeSlice = src.duplicate();
|
|
|
- int remaining = Math.min(buf.remaining(), writeSlice.remaining());
|
|
|
- writeSlice.limit(writeSlice.position() + remaining);
|
|
|
- buf.put(writeSlice);
|
|
|
- return remaining;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/* This is a used by regular read() and handles ChecksumExceptions.
|
|
|
* name readBuffer() is chosen to imply similarity to readBuffer() in
|
|
|
* ChecksumFileSystem
|
|
|
*/
|
|
|
- private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
|
|
|
+ private synchronized int readBuffer(ReaderStrategy reader, int len,
|
|
|
CorruptedBlocks corruptedBlocks)
|
|
|
throws IOException {
|
|
|
IOException ioe;
|
|
@@ -892,7 +700,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
while (true) {
|
|
|
// retry as many times as seekToNewSource allows.
|
|
|
try {
|
|
|
- return reader.doRead(blockReader, off, len);
|
|
|
+ return reader.readFromBlock(blockReader, len);
|
|
|
} catch ( ChecksumException ce ) {
|
|
|
DFSClient.LOG.warn("Found Checksum error for "
|
|
|
+ getCurrentBlock() + " from " + currentNode
|
|
@@ -927,13 +735,14 @@ public class DFSInputStream extends FSInputStream
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- protected synchronized int readWithStrategy(ReaderStrategy strategy, int off,
|
|
|
- int len) throws IOException {
|
|
|
+ protected synchronized int readWithStrategy(ReaderStrategy strategy)
|
|
|
+ throws IOException {
|
|
|
dfsClient.checkOpen();
|
|
|
if (closed.get()) {
|
|
|
throw new IOException("Stream closed");
|
|
|
}
|
|
|
|
|
|
+ int len = strategy.getTargetLength();
|
|
|
CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
|
|
|
failures = 0;
|
|
|
if (pos < getFileLength()) {
|
|
@@ -952,7 +761,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
locatedBlocks.getFileLength() - pos);
|
|
|
}
|
|
|
}
|
|
|
- int result = readBuffer(strategy, off, realLen, corruptedBlocks);
|
|
|
+ int result = readBuffer(strategy, realLen, corruptedBlocks);
|
|
|
|
|
|
if (result >= 0) {
|
|
|
pos += result;
|
|
@@ -994,11 +803,12 @@ public class DFSInputStream extends FSInputStream
|
|
|
if (len == 0) {
|
|
|
return 0;
|
|
|
}
|
|
|
- ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
|
|
|
+ ReaderStrategy byteArrayReader =
|
|
|
+ new ByteArrayStrategy(buf, off, len, readStatistics, dfsClient);
|
|
|
try (TraceScope scope =
|
|
|
dfsClient.newReaderTraceScope("DFSInputStream#byteArrayRead",
|
|
|
src, getPos(), len)) {
|
|
|
- int retLen = readWithStrategy(byteArrayReader, off, len);
|
|
|
+ int retLen = readWithStrategy(byteArrayReader);
|
|
|
if (retLen < len) {
|
|
|
dfsClient.addRetLenToReaderScope(scope, retLen);
|
|
|
}
|
|
@@ -1008,12 +818,13 @@ public class DFSInputStream extends FSInputStream
|
|
|
|
|
|
@Override
|
|
|
public synchronized int read(final ByteBuffer buf) throws IOException {
|
|
|
- ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
|
|
|
+ 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, 0, reqLen);
|
|
|
+ int retLen = readWithStrategy(byteBufferReader);
|
|
|
if (retLen < reqLen) {
|
|
|
dfsClient.addRetLenToReaderScope(scope, retLen);
|
|
|
}
|
|
@@ -1221,7 +1032,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
reader = getBlockReader(block, startInBlk, len, datanode.addr,
|
|
|
datanode.storageType, datanode.info);
|
|
|
int nread = reader.readAll(buf, offset, len);
|
|
|
- updateReadStatistics(readStatistics, nread, reader);
|
|
|
+ IOUtilsClient.updateReadStatistics(readStatistics, nread, reader);
|
|
|
dfsClient.updateFileSystemReadStats(
|
|
|
reader.getNetworkDistance(), nread);
|
|
|
if (nread != len) {
|
|
@@ -1721,18 +1532,14 @@ public class DFSInputStream extends FSInputStream
|
|
|
* Get statistics about the reads which this DFSInputStream has done.
|
|
|
*/
|
|
|
public ReadStatistics getReadStatistics() {
|
|
|
- synchronized(infoLock) {
|
|
|
- return new ReadStatistics(readStatistics);
|
|
|
- }
|
|
|
+ return new ReadStatistics(readStatistics);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Clear statistics about the reads which this DFSInputStream has done.
|
|
|
*/
|
|
|
public void clearReadStatistics() {
|
|
|
- synchronized(infoLock) {
|
|
|
- readStatistics.clear();
|
|
|
- }
|
|
|
+ readStatistics.clear();
|
|
|
}
|
|
|
|
|
|
public FileEncryptionInfo getFileEncryptionInfo() {
|
|
@@ -1759,7 +1566,8 @@ public class DFSInputStream extends FSInputStream
|
|
|
throws IOException {
|
|
|
synchronized (infoLock) {
|
|
|
this.cachingStrategy =
|
|
|
- new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
|
|
|
+ new CachingStrategy.Builder(this.cachingStrategy).
|
|
|
+ setReadahead(readahead).build();
|
|
|
}
|
|
|
closeCurrentBlockReaders();
|
|
|
}
|
|
@@ -1769,7 +1577,8 @@ public class DFSInputStream extends FSInputStream
|
|
|
throws IOException {
|
|
|
synchronized (infoLock) {
|
|
|
this.cachingStrategy =
|
|
|
- new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
|
|
|
+ new CachingStrategy.Builder(this.cachingStrategy).
|
|
|
+ setDropBehind(dropBehind).build();
|
|
|
}
|
|
|
closeCurrentBlockReaders();
|
|
|
}
|
|
@@ -1883,9 +1692,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
buffer.position((int)blockPos);
|
|
|
buffer.limit((int)(blockPos + length));
|
|
|
getExtendedReadBuffers().put(buffer, clientMmap);
|
|
|
- synchronized (infoLock) {
|
|
|
- readStatistics.addZeroCopyBytes(length);
|
|
|
- }
|
|
|
+ readStatistics.addZeroCopyBytes(length);
|
|
|
DFSClient.LOG.debug("readZeroCopy read {} bytes from offset {} via the "
|
|
|
+ "zero-copy read path. blockEnd = {}", length, curPos, blockEnd);
|
|
|
success = true;
|