|
@@ -51,8 +51,6 @@ 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.core.TraceScope;
|
|
|
|
-import org.apache.htrace.core.Tracer;
|
|
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
@@ -184,7 +182,6 @@ 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.
|
|
@@ -193,8 +190,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,
|
|
|
|
- Tracer tracer) throws IOException {
|
|
|
|
|
|
+ long startOffset, long length, StorageType storageType)
|
|
|
|
+ throws IOException {
|
|
final ShortCircuitConf scConf = conf.getShortCircuitConf();
|
|
final ShortCircuitConf scConf = conf.getShortCircuitConf();
|
|
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
|
|
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
|
|
.getIpcPort());
|
|
.getIpcPort());
|
|
@@ -239,11 +236,10 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|
long firstChunkOffset = startOffset
|
|
long firstChunkOffset = startOffset
|
|
- (startOffset % checksum.getBytesPerChecksum());
|
|
- (startOffset % checksum.getBytesPerChecksum());
|
|
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
|
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
|
- startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn,
|
|
|
|
- tracer);
|
|
|
|
|
|
+ startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn);
|
|
} else {
|
|
} else {
|
|
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
|
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
|
- startOffset, dataIn, tracer);
|
|
|
|
|
|
+ startOffset, dataIn);
|
|
}
|
|
}
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
// remove from cache
|
|
// remove from cache
|
|
@@ -320,17 +316,17 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|
}
|
|
}
|
|
|
|
|
|
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
|
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,
|
|
this(conf, hdfsfile, block, startOffset,
|
|
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
|
|
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
|
|
- dataIn, startOffset, null, tracer);
|
|
|
|
|
|
+ dataIn, startOffset, null);
|
|
}
|
|
}
|
|
|
|
|
|
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
|
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
|
ExtendedBlock block, long startOffset, DataChecksum checksum,
|
|
ExtendedBlock block, long startOffset, DataChecksum checksum,
|
|
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
|
|
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
|
|
- FileInputStream checksumIn, Tracer tracer) throws IOException {
|
|
|
|
|
|
+ FileInputStream checksumIn) throws IOException {
|
|
this.filename = hdfsfile;
|
|
this.filename = hdfsfile;
|
|
this.checksum = checksum;
|
|
this.checksum = checksum;
|
|
this.verifyChecksum = verifyChecksum;
|
|
this.verifyChecksum = verifyChecksum;
|
|
@@ -369,7 +365,6 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|
bufferPool.returnBuffer(checksumBuff);
|
|
bufferPool.returnBuffer(checksumBuff);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- this.tracer = tracer;
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -377,23 +372,20 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|
*/
|
|
*/
|
|
private int fillBuffer(FileInputStream stream, ByteBuffer buf)
|
|
private int fillBuffer(FileInputStream stream, ByteBuffer buf)
|
|
throws IOException {
|
|
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
|
|
//EOF
|
|
return bytesRead;
|
|
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;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|