|
@@ -131,10 +131,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
|
|
|
public static class ReadStatistics {
|
|
public static class ReadStatistics {
|
|
public ReadStatistics() {
|
|
public ReadStatistics() {
|
|
- this.totalBytesRead = 0;
|
|
|
|
- this.totalLocalBytesRead = 0;
|
|
|
|
- this.totalShortCircuitBytesRead = 0;
|
|
|
|
- this.totalZeroCopyBytesRead = 0;
|
|
|
|
|
|
+ clear();
|
|
}
|
|
}
|
|
|
|
|
|
public ReadStatistics(ReadStatistics rhs) {
|
|
public ReadStatistics(ReadStatistics rhs) {
|
|
@@ -203,6 +200,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
this.totalShortCircuitBytesRead += amt;
|
|
this.totalShortCircuitBytesRead += amt;
|
|
this.totalZeroCopyBytesRead += amt;
|
|
this.totalZeroCopyBytesRead += amt;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ void clear() {
|
|
|
|
+ this.totalBytesRead = 0;
|
|
|
|
+ this.totalLocalBytesRead = 0;
|
|
|
|
+ this.totalShortCircuitBytesRead = 0;
|
|
|
|
+ this.totalZeroCopyBytesRead = 0;
|
|
|
|
+ }
|
|
|
|
|
|
private long totalBytesRead;
|
|
private long totalBytesRead;
|
|
|
|
|
|
@@ -412,7 +416,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
/**
|
|
/**
|
|
* Return collection of blocks that has already been located.
|
|
* Return collection of blocks that has already been located.
|
|
*/
|
|
*/
|
|
- public synchronized List<LocatedBlock> getAllBlocks() throws IOException {
|
|
|
|
|
|
+ public List<LocatedBlock> getAllBlocks() throws IOException {
|
|
return getBlockRange(0, getFileLength());
|
|
return getBlockRange(0, getFileLength());
|
|
}
|
|
}
|
|
|
|
|
|
@@ -700,26 +704,28 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
* strategy-agnostic.
|
|
* strategy-agnostic.
|
|
*/
|
|
*/
|
|
private interface ReaderStrategy {
|
|
private interface ReaderStrategy {
|
|
- public int doRead(BlockReader blockReader, int off, int len,
|
|
|
|
- ReadStatistics readStatistics) throws ChecksumException, IOException;
|
|
|
|
|
|
+ public int doRead(BlockReader blockReader, int off, int len)
|
|
|
|
+ throws ChecksumException, IOException;
|
|
}
|
|
}
|
|
|
|
|
|
- private static void updateReadStatistics(ReadStatistics readStatistics,
|
|
|
|
|
|
+ private void updateReadStatistics(ReadStatistics readStatistics,
|
|
int nRead, BlockReader blockReader) {
|
|
int nRead, BlockReader blockReader) {
|
|
if (nRead <= 0) return;
|
|
if (nRead <= 0) return;
|
|
- if (blockReader.isShortCircuit()) {
|
|
|
|
- readStatistics.addShortCircuitBytes(nRead);
|
|
|
|
- } else if (blockReader.isLocal()) {
|
|
|
|
- readStatistics.addLocalBytes(nRead);
|
|
|
|
- } else {
|
|
|
|
- readStatistics.addRemoteBytes(nRead);
|
|
|
|
|
|
+ synchronized(infoLock) {
|
|
|
|
+ if (blockReader.isShortCircuit()) {
|
|
|
|
+ readStatistics.addShortCircuitBytes(nRead);
|
|
|
|
+ } else if (blockReader.isLocal()) {
|
|
|
|
+ readStatistics.addLocalBytes(nRead);
|
|
|
|
+ } else {
|
|
|
|
+ readStatistics.addRemoteBytes(nRead);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Used to read bytes into a byte[]
|
|
* Used to read bytes into a byte[]
|
|
*/
|
|
*/
|
|
- private static class ByteArrayStrategy implements ReaderStrategy {
|
|
|
|
|
|
+ private class ByteArrayStrategy implements ReaderStrategy {
|
|
final byte[] buf;
|
|
final byte[] buf;
|
|
|
|
|
|
public ByteArrayStrategy(byte[] buf) {
|
|
public ByteArrayStrategy(byte[] buf) {
|
|
@@ -727,26 +733,26 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public int doRead(BlockReader blockReader, int off, int len,
|
|
|
|
- ReadStatistics readStatistics) throws ChecksumException, IOException {
|
|
|
|
- int nRead = blockReader.read(buf, off, len);
|
|
|
|
- updateReadStatistics(readStatistics, nRead, blockReader);
|
|
|
|
- return nRead;
|
|
|
|
|
|
+ public int doRead(BlockReader blockReader, int off, int len)
|
|
|
|
+ throws ChecksumException, IOException {
|
|
|
|
+ int nRead = blockReader.read(buf, off, len);
|
|
|
|
+ updateReadStatistics(readStatistics, nRead, blockReader);
|
|
|
|
+ return nRead;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Used to read bytes into a user-supplied ByteBuffer
|
|
* Used to read bytes into a user-supplied ByteBuffer
|
|
*/
|
|
*/
|
|
- private static class ByteBufferStrategy implements ReaderStrategy {
|
|
|
|
|
|
+ private class ByteBufferStrategy implements ReaderStrategy {
|
|
final ByteBuffer buf;
|
|
final ByteBuffer buf;
|
|
ByteBufferStrategy(ByteBuffer buf) {
|
|
ByteBufferStrategy(ByteBuffer buf) {
|
|
this.buf = buf;
|
|
this.buf = buf;
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public int doRead(BlockReader blockReader, int off, int len,
|
|
|
|
- ReadStatistics readStatistics) throws ChecksumException, IOException {
|
|
|
|
|
|
+ public int doRead(BlockReader blockReader, int off, int len)
|
|
|
|
+ throws ChecksumException, IOException {
|
|
int oldpos = buf.position();
|
|
int oldpos = buf.position();
|
|
int oldlimit = buf.limit();
|
|
int oldlimit = buf.limit();
|
|
boolean success = false;
|
|
boolean success = false;
|
|
@@ -785,7 +791,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
while (true) {
|
|
while (true) {
|
|
// retry as many times as seekToNewSource allows.
|
|
// retry as many times as seekToNewSource allows.
|
|
try {
|
|
try {
|
|
- return reader.doRead(blockReader, off, len, readStatistics);
|
|
|
|
|
|
+ return reader.doRead(blockReader, off, len);
|
|
} catch ( ChecksumException ce ) {
|
|
} catch ( ChecksumException ce ) {
|
|
DFSClient.LOG.warn("Found Checksum error for "
|
|
DFSClient.LOG.warn("Found Checksum error for "
|
|
+ getCurrentBlock() + " from " + currentNode
|
|
+ getCurrentBlock() + " from " + currentNode
|
|
@@ -1612,8 +1618,19 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
/**
|
|
/**
|
|
* Get statistics about the reads which this DFSInputStream has done.
|
|
* Get statistics about the reads which this DFSInputStream has done.
|
|
*/
|
|
*/
|
|
- public synchronized ReadStatistics getReadStatistics() {
|
|
|
|
- return new ReadStatistics(readStatistics);
|
|
|
|
|
|
+ public ReadStatistics getReadStatistics() {
|
|
|
|
+ synchronized(infoLock) {
|
|
|
|
+ return new ReadStatistics(readStatistics);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Clear statistics about the reads which this DFSInputStream has done.
|
|
|
|
+ */
|
|
|
|
+ public void clearReadStatistics() {
|
|
|
|
+ synchronized(infoLock) {
|
|
|
|
+ readStatistics.clear();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
public FileEncryptionInfo getFileEncryptionInfo() {
|
|
public FileEncryptionInfo getFileEncryptionInfo() {
|
|
@@ -1775,7 +1792,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
buffer.position((int)blockPos);
|
|
buffer.position((int)blockPos);
|
|
buffer.limit((int)(blockPos + length));
|
|
buffer.limit((int)(blockPos + length));
|
|
extendedReadBuffers.put(buffer, clientMmap);
|
|
extendedReadBuffers.put(buffer, clientMmap);
|
|
- readStatistics.addZeroCopyBytes(length);
|
|
|
|
|
|
+ synchronized (infoLock) {
|
|
|
|
+ readStatistics.addZeroCopyBytes(length);
|
|
|
|
+ }
|
|
if (DFSClient.LOG.isDebugEnabled()) {
|
|
if (DFSClient.LOG.isDebugEnabled()) {
|
|
DFSClient.LOG.debug("readZeroCopy read " + length +
|
|
DFSClient.LOG.debug("readZeroCopy read " + length +
|
|
" bytes from offset " + curPos + " via the zero-copy read " +
|
|
" bytes from offset " + curPos + " via the zero-copy read " +
|