HDFS-4352. Encapsulate arguments to BlockReaderFactory in a class. Contributed by Colin Patrick McCabe.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1428729 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
228de3c987
commit
32052a1e3a
|
@ -177,6 +177,9 @@ Trunk (Unreleased)
|
||||||
HDFS-4346. Add SequentialNumber as a base class for INodeId and
|
HDFS-4346. Add SequentialNumber as a base class for INodeId and
|
||||||
GenerationStamp. (szetszwo)
|
GenerationStamp. (szetszwo)
|
||||||
|
|
||||||
|
HDFS-4352. Encapsulate arguments to BlockReaderFactory in a class
|
||||||
|
(Colin Patrick McCabe via todd)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
|
@ -22,7 +22,6 @@ import java.net.InetSocketAddress;
|
||||||
import java.net.Socket;
|
import java.net.Socket;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
|
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
|
||||||
|
@ -40,71 +39,150 @@ import org.apache.hadoop.security.token.Token;
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class BlockReaderFactory {
|
public class BlockReaderFactory {
|
||||||
/**
|
/**
|
||||||
* @see #newBlockReader(Conf, Socket, String, ExtendedBlock, Token, long, long, int, boolean, String)
|
* Parameters for creating a BlockReader.
|
||||||
|
*
|
||||||
|
* Before you add something to here: think about whether it's already included
|
||||||
|
* in Conf (or should be).
|
||||||
*/
|
*/
|
||||||
public static BlockReader newBlockReader(
|
@InterfaceAudience.Private
|
||||||
Configuration conf,
|
public static class Params {
|
||||||
Socket sock, String file,
|
private final Conf conf;
|
||||||
ExtendedBlock block, Token<BlockTokenIdentifier> blockToken,
|
private Socket socket = null;
|
||||||
long startOffset, long len, DataEncryptionKey encryptionKey)
|
private String file = null;
|
||||||
throws IOException {
|
private ExtendedBlock block = null;
|
||||||
int bufferSize = conf.getInt(DFSConfigKeys.IO_FILE_BUFFER_SIZE_KEY,
|
private Token<BlockTokenIdentifier> blockToken = null;
|
||||||
DFSConfigKeys.IO_FILE_BUFFER_SIZE_DEFAULT);
|
private long startOffset = 0;
|
||||||
return newBlockReader(new Conf(conf),
|
private long len = -1;
|
||||||
sock, file, block, blockToken, startOffset,
|
private int bufferSize;
|
||||||
len, bufferSize, true, "", encryptionKey, null);
|
private boolean verifyChecksum = true;
|
||||||
|
private boolean shortCircuitLocalReads = false;
|
||||||
|
private String clientName = "";
|
||||||
|
private DataEncryptionKey encryptionKey = null;
|
||||||
|
private IOStreamPair ioStreamPair = null;
|
||||||
|
|
||||||
|
public Params(Conf conf) {
|
||||||
|
this.conf = conf;
|
||||||
|
this.bufferSize = conf.ioBufferSize;
|
||||||
|
}
|
||||||
|
public Conf getConf() {
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
public Socket getSocket() {
|
||||||
|
return socket;
|
||||||
|
}
|
||||||
|
public Params setSocket(Socket socket) {
|
||||||
|
this.socket = socket;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public String getFile() {
|
||||||
|
return file;
|
||||||
|
}
|
||||||
|
public Params setFile(String file) {
|
||||||
|
this.file = file;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public ExtendedBlock getBlock() {
|
||||||
|
return block;
|
||||||
|
}
|
||||||
|
public Params setBlock(ExtendedBlock block) {
|
||||||
|
this.block = block;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Token<BlockTokenIdentifier> getBlockToken() {
|
||||||
|
return blockToken;
|
||||||
|
}
|
||||||
|
public Params setBlockToken(Token<BlockTokenIdentifier> blockToken) {
|
||||||
|
this.blockToken = blockToken;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public long getStartOffset() {
|
||||||
|
return startOffset;
|
||||||
|
}
|
||||||
|
public Params setStartOffset(long startOffset) {
|
||||||
|
this.startOffset = startOffset;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public long getLen() {
|
||||||
|
return len;
|
||||||
|
}
|
||||||
|
public Params setLen(long len) {
|
||||||
|
this.len = len;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public int getBufferSize() {
|
||||||
|
return bufferSize;
|
||||||
|
}
|
||||||
|
public Params setBufferSize(int bufferSize) {
|
||||||
|
this.bufferSize = bufferSize;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public boolean getVerifyChecksum() {
|
||||||
|
return verifyChecksum;
|
||||||
|
}
|
||||||
|
public Params setVerifyChecksum(boolean verifyChecksum) {
|
||||||
|
this.verifyChecksum = verifyChecksum;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public boolean getShortCircuitLocalReads() {
|
||||||
|
return shortCircuitLocalReads;
|
||||||
|
}
|
||||||
|
public Params setShortCircuitLocalReads(boolean on) {
|
||||||
|
this.shortCircuitLocalReads = on;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public String getClientName() {
|
||||||
|
return clientName;
|
||||||
|
}
|
||||||
|
public Params setClientName(String clientName) {
|
||||||
|
this.clientName = clientName;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Params setEncryptionKey(DataEncryptionKey encryptionKey) {
|
||||||
|
this.encryptionKey = encryptionKey;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public DataEncryptionKey getEncryptionKey() {
|
||||||
|
return encryptionKey;
|
||||||
|
}
|
||||||
|
public IOStreamPair getIoStreamPair() {
|
||||||
|
return ioStreamPair;
|
||||||
|
}
|
||||||
|
public Params setIoStreamPair(IOStreamPair ioStreamPair) {
|
||||||
|
this.ioStreamPair = ioStreamPair;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new BlockReader specifically to satisfy a read.
|
* Create a new BlockReader specifically to satisfy a read.
|
||||||
* This method also sends the OP_READ_BLOCK request.
|
* This method also sends the OP_READ_BLOCK request.
|
||||||
*
|
*
|
||||||
* @param conf the DFSClient configuration
|
* @param params The parameters
|
||||||
* @param sock An established Socket to the DN. The BlockReader will not close it normally
|
*
|
||||||
* @param file File location
|
* @return New BlockReader instance
|
||||||
* @param block The block object
|
* @throws IOException If there was an error creating the BlockReader
|
||||||
* @param blockToken The block token for security
|
|
||||||
* @param startOffset The read offset, relative to block head
|
|
||||||
* @param len The number of bytes to read
|
|
||||||
* @param bufferSize The IO buffer size (not the client buffer size)
|
|
||||||
* @param verifyChecksum Whether to verify checksum
|
|
||||||
* @param clientName Client name
|
|
||||||
* @return New BlockReader instance, or null on error.
|
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public static BlockReader newBlockReader(
|
public static BlockReader newBlockReader(Params params) throws IOException {
|
||||||
Conf conf,
|
if (params.getConf().useLegacyBlockReader) {
|
||||||
Socket sock, String file,
|
if (params.getEncryptionKey() != null) {
|
||||||
ExtendedBlock block,
|
|
||||||
Token<BlockTokenIdentifier> blockToken,
|
|
||||||
long startOffset, long len,
|
|
||||||
int bufferSize, boolean verifyChecksum,
|
|
||||||
String clientName,
|
|
||||||
DataEncryptionKey encryptionKey,
|
|
||||||
IOStreamPair ioStreams)
|
|
||||||
throws IOException {
|
|
||||||
|
|
||||||
if (conf.useLegacyBlockReader) {
|
|
||||||
if (encryptionKey != null) {
|
|
||||||
throw new RuntimeException("Encryption is not supported with the legacy block reader.");
|
throw new RuntimeException("Encryption is not supported with the legacy block reader.");
|
||||||
}
|
}
|
||||||
return RemoteBlockReader.newBlockReader(
|
return RemoteBlockReader.newBlockReader(params);
|
||||||
sock, file, block, blockToken, startOffset, len, bufferSize, verifyChecksum, clientName);
|
|
||||||
} else {
|
} else {
|
||||||
if (ioStreams == null) {
|
Socket sock = params.getSocket();
|
||||||
ioStreams = new IOStreamPair(NetUtils.getInputStream(sock),
|
if (params.getIoStreamPair() == null) {
|
||||||
NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT));
|
params.setIoStreamPair(new IOStreamPair(NetUtils.getInputStream(sock),
|
||||||
if (encryptionKey != null) {
|
NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT)));
|
||||||
|
if (params.getEncryptionKey() != null) {
|
||||||
IOStreamPair encryptedStreams =
|
IOStreamPair encryptedStreams =
|
||||||
DataTransferEncryptor.getEncryptedStreams(
|
DataTransferEncryptor.getEncryptedStreams(
|
||||||
ioStreams.out, ioStreams.in, encryptionKey);
|
params.getIoStreamPair().out, params.getIoStreamPair().in,
|
||||||
ioStreams = encryptedStreams;
|
params.getEncryptionKey());
|
||||||
|
params.setIoStreamPair(encryptedStreams);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
return RemoteBlockReader2.newBlockReader(params);
|
||||||
return RemoteBlockReader2.newBlockReader(
|
|
||||||
sock, file, block, blockToken, startOffset, len, bufferSize,
|
|
||||||
verifyChecksum, clientName, encryptionKey, ioStreams);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -200,7 +200,8 @@ public class DFSClient implements java.io.Closeable {
|
||||||
/**
|
/**
|
||||||
* DFSClient configuration
|
* DFSClient configuration
|
||||||
*/
|
*/
|
||||||
static class Conf {
|
@InterfaceAudience.Private
|
||||||
|
public static class Conf {
|
||||||
final int maxFailoverAttempts;
|
final int maxFailoverAttempts;
|
||||||
final int failoverSleepBaseMillis;
|
final int failoverSleepBaseMillis;
|
||||||
final int failoverSleepMaxMillis;
|
final int failoverSleepMaxMillis;
|
||||||
|
@ -228,7 +229,7 @@ public class DFSClient implements java.io.Closeable {
|
||||||
final int getFileBlockStorageLocationsNumThreads;
|
final int getFileBlockStorageLocationsNumThreads;
|
||||||
final int getFileBlockStorageLocationsTimeout;
|
final int getFileBlockStorageLocationsTimeout;
|
||||||
|
|
||||||
Conf(Configuration conf) {
|
public Conf(Configuration conf) {
|
||||||
maxFailoverAttempts = conf.getInt(
|
maxFailoverAttempts = conf.getInt(
|
||||||
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
|
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
|
||||||
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
|
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
|
||||||
|
|
|
@ -934,15 +934,15 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
||||||
|
|
||||||
try {
|
try {
|
||||||
// The OP_READ_BLOCK request is sent as we make the BlockReader
|
// The OP_READ_BLOCK request is sent as we make the BlockReader
|
||||||
BlockReader reader =
|
BlockReader reader = BlockReaderFactory.
|
||||||
BlockReaderFactory.newBlockReader(dfsClient.getConf(),
|
newBlockReader(new BlockReaderFactory.Params(dfsClient.getConf()).
|
||||||
sock, file, block,
|
setFile(file).setBlock(block).setBlockToken(blockToken).
|
||||||
blockToken,
|
setStartOffset(startOffset).setLen(len).
|
||||||
startOffset, len,
|
setBufferSize(bufferSize).setVerifyChecksum(verifyChecksum).
|
||||||
bufferSize, verifyChecksum,
|
setClientName(clientName).
|
||||||
clientName,
|
setEncryptionKey(dfsClient.getDataEncryptionKey()).
|
||||||
dfsClient.getDataEncryptionKey(),
|
setIoStreamPair(sockAndStreams == null ? null : sockAndStreams.ioStreams).
|
||||||
sockAndStreams == null ? null : sockAndStreams.ioStreams);
|
setSocket(sock));
|
||||||
return reader;
|
return reader;
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
// Our socket is no good.
|
// Our socket is no good.
|
||||||
|
|
|
@ -349,13 +349,6 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
||||||
checksumSize = this.checksum.getChecksumSize();
|
checksumSize = this.checksum.getChecksumSize();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static RemoteBlockReader newBlockReader(Socket sock, String file,
|
|
||||||
ExtendedBlock block, Token<BlockTokenIdentifier> blockToken,
|
|
||||||
long startOffset, long len, int bufferSize) throws IOException {
|
|
||||||
return newBlockReader(sock, file, block, blockToken, startOffset,
|
|
||||||
len, bufferSize, true, "");
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new BlockReader specifically to satisfy a read.
|
* Create a new BlockReader specifically to satisfy a read.
|
||||||
* This method also sends the OP_READ_BLOCK request.
|
* This method also sends the OP_READ_BLOCK request.
|
||||||
|
@ -371,29 +364,26 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
||||||
* @param clientName Client name
|
* @param clientName Client name
|
||||||
* @return New BlockReader instance, or null on error.
|
* @return New BlockReader instance, or null on error.
|
||||||
*/
|
*/
|
||||||
public static RemoteBlockReader newBlockReader( Socket sock, String file,
|
public static RemoteBlockReader newBlockReader(BlockReaderFactory.Params params)
|
||||||
ExtendedBlock block,
|
throws IOException {
|
||||||
Token<BlockTokenIdentifier> blockToken,
|
|
||||||
long startOffset, long len,
|
|
||||||
int bufferSize, boolean verifyChecksum,
|
|
||||||
String clientName)
|
|
||||||
throws IOException {
|
|
||||||
// in and out will be closed when sock is closed (by the caller)
|
// in and out will be closed when sock is closed (by the caller)
|
||||||
|
Socket sock = params.getSocket();
|
||||||
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
||||||
NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT)));
|
NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT)));
|
||||||
new Sender(out).readBlock(block, blockToken, clientName, startOffset, len);
|
new Sender(out).readBlock(params.getBlock(), params.getBlockToken(),
|
||||||
|
params.getClientName(), params.getStartOffset(), params.getLen());
|
||||||
|
|
||||||
//
|
//
|
||||||
// Get bytes in block, set streams
|
// Get bytes in block, set streams
|
||||||
//
|
//
|
||||||
|
|
||||||
DataInputStream in = new DataInputStream(
|
DataInputStream in = new DataInputStream(
|
||||||
new BufferedInputStream(NetUtils.getInputStream(sock),
|
new BufferedInputStream(NetUtils.getInputStream(sock),
|
||||||
bufferSize));
|
params.getBufferSize()));
|
||||||
|
|
||||||
BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
|
BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
|
||||||
vintPrefixed(in));
|
vintPrefixed(in));
|
||||||
RemoteBlockReader2.checkSuccess(status, sock, block, file);
|
RemoteBlockReader2.checkSuccess(status, sock, params.getBlock(),
|
||||||
|
params.getFile());
|
||||||
ReadOpChecksumInfoProto checksumInfo =
|
ReadOpChecksumInfoProto checksumInfo =
|
||||||
status.getReadOpChecksumInfo();
|
status.getReadOpChecksumInfo();
|
||||||
DataChecksum checksum = DataTransferProtoUtil.fromProto(
|
DataChecksum checksum = DataTransferProtoUtil.fromProto(
|
||||||
|
@ -403,15 +393,16 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
||||||
// Read the first chunk offset.
|
// Read the first chunk offset.
|
||||||
long firstChunkOffset = checksumInfo.getChunkOffset();
|
long firstChunkOffset = checksumInfo.getChunkOffset();
|
||||||
|
|
||||||
if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
|
if ( firstChunkOffset < 0 || firstChunkOffset > params.getStartOffset() ||
|
||||||
firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
|
firstChunkOffset <= (params.getStartOffset() - checksum.getBytesPerChecksum())) {
|
||||||
throw new IOException("BlockReader: error in first chunk offset (" +
|
throw new IOException("BlockReader: error in first chunk offset (" +
|
||||||
firstChunkOffset + ") startOffset is " +
|
firstChunkOffset + ") startOffset is " +
|
||||||
startOffset + " for file " + file);
|
params.getStartOffset() + " for file " + params.getFile());
|
||||||
}
|
}
|
||||||
|
|
||||||
return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
|
return new RemoteBlockReader(params.getFile(), params.getBlock().getBlockPoolId(),
|
||||||
in, checksum, verifyChecksum, startOffset, firstChunkOffset, len, sock);
|
params.getBlock().getBlockId(), in, checksum, params.getVerifyChecksum(),
|
||||||
|
params.getStartOffset(), firstChunkOffset, params.getLen(), sock);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -246,24 +246,22 @@ public class RemoteBlockReader2 implements BlockReader {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected RemoteBlockReader2(String file, String bpid, long blockId,
|
protected RemoteBlockReader2(BlockReaderFactory.Params params,
|
||||||
ReadableByteChannel in, DataChecksum checksum, boolean verifyChecksum,
|
DataChecksum checksum, long firstChunkOffset, ReadableByteChannel in) {
|
||||||
long startOffset, long firstChunkOffset, long bytesToRead, Socket dnSock,
|
|
||||||
IOStreamPair ioStreams) {
|
|
||||||
// Path is used only for printing block and file information in debug
|
// Path is used only for printing block and file information in debug
|
||||||
this.dnSock = dnSock;
|
this.dnSock = params.getSocket();
|
||||||
this.ioStreams = ioStreams;
|
this.ioStreams = params.getIoStreamPair();
|
||||||
this.in = in;
|
this.in = in;
|
||||||
this.checksum = checksum;
|
this.checksum = checksum;
|
||||||
this.verifyChecksum = verifyChecksum;
|
this.verifyChecksum = params.getVerifyChecksum();
|
||||||
this.startOffset = Math.max( startOffset, 0 );
|
this.startOffset = Math.max( params.getStartOffset(), 0 );
|
||||||
this.filename = file;
|
this.filename = params.getFile();
|
||||||
|
|
||||||
// The total number of bytes that we need to transfer from the DN is
|
// The total number of bytes that we need to transfer from the DN is
|
||||||
// the amount that the user wants (bytesToRead), plus the padding at
|
// the amount that the user wants (bytesToRead), plus the padding at
|
||||||
// the beginning in order to chunk-align. Note that the DN may elect
|
// the beginning in order to chunk-align. Note that the DN may elect
|
||||||
// to send more than this amount if the read starts/ends mid-chunk.
|
// to send more than this amount if the read starts/ends mid-chunk.
|
||||||
this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
|
this.bytesNeededToFinish = params.getLen() + (startOffset - firstChunkOffset);
|
||||||
bytesPerChecksum = this.checksum.getBytesPerChecksum();
|
bytesPerChecksum = this.checksum.getBytesPerChecksum();
|
||||||
checksumSize = this.checksum.getChecksumSize();
|
checksumSize = this.checksum.getChecksumSize();
|
||||||
}
|
}
|
||||||
|
@ -373,16 +371,9 @@ public class RemoteBlockReader2 implements BlockReader {
|
||||||
* @param clientName Client name
|
* @param clientName Client name
|
||||||
* @return New BlockReader instance, or null on error.
|
* @return New BlockReader instance, or null on error.
|
||||||
*/
|
*/
|
||||||
public static BlockReader newBlockReader(Socket sock, String file,
|
public static BlockReader newBlockReader(BlockReaderFactory.Params params)
|
||||||
ExtendedBlock block,
|
|
||||||
Token<BlockTokenIdentifier> blockToken,
|
|
||||||
long startOffset, long len,
|
|
||||||
int bufferSize, boolean verifyChecksum,
|
|
||||||
String clientName,
|
|
||||||
DataEncryptionKey encryptionKey,
|
|
||||||
IOStreamPair ioStreams)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
IOStreamPair ioStreams = params.getIoStreamPair();
|
||||||
ReadableByteChannel ch;
|
ReadableByteChannel ch;
|
||||||
if (ioStreams.in instanceof SocketInputWrapper) {
|
if (ioStreams.in instanceof SocketInputWrapper) {
|
||||||
ch = ((SocketInputWrapper)ioStreams.in).getReadableByteChannel();
|
ch = ((SocketInputWrapper)ioStreams.in).getReadableByteChannel();
|
||||||
|
@ -393,7 +384,8 @@ public class RemoteBlockReader2 implements BlockReader {
|
||||||
// in and out will be closed when sock is closed (by the caller)
|
// in and out will be closed when sock is closed (by the caller)
|
||||||
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
||||||
ioStreams.out));
|
ioStreams.out));
|
||||||
new Sender(out).readBlock(block, blockToken, clientName, startOffset, len);
|
new Sender(out).readBlock(params.getBlock(), params.getBlockToken(),
|
||||||
|
params.getClientName(), params.getStartOffset(), params.getLen());
|
||||||
|
|
||||||
//
|
//
|
||||||
// Get bytes in block
|
// Get bytes in block
|
||||||
|
@ -402,7 +394,8 @@ public class RemoteBlockReader2 implements BlockReader {
|
||||||
|
|
||||||
BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
|
BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
|
||||||
vintPrefixed(in));
|
vintPrefixed(in));
|
||||||
checkSuccess(status, sock, block, file);
|
checkSuccess(status, params.getSocket(), params.getBlock(),
|
||||||
|
params.getFile());
|
||||||
ReadOpChecksumInfoProto checksumInfo =
|
ReadOpChecksumInfoProto checksumInfo =
|
||||||
status.getReadOpChecksumInfo();
|
status.getReadOpChecksumInfo();
|
||||||
DataChecksum checksum = DataTransferProtoUtil.fromProto(
|
DataChecksum checksum = DataTransferProtoUtil.fromProto(
|
||||||
|
@ -412,16 +405,14 @@ public class RemoteBlockReader2 implements BlockReader {
|
||||||
// Read the first chunk offset.
|
// Read the first chunk offset.
|
||||||
long firstChunkOffset = checksumInfo.getChunkOffset();
|
long firstChunkOffset = checksumInfo.getChunkOffset();
|
||||||
|
|
||||||
if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
|
if ( firstChunkOffset < 0 || firstChunkOffset > params.getStartOffset() ||
|
||||||
firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
|
firstChunkOffset <= (params.getStartOffset() - checksum.getBytesPerChecksum())) {
|
||||||
throw new IOException("BlockReader: error in first chunk offset (" +
|
throw new IOException("BlockReader: error in first chunk offset (" +
|
||||||
firstChunkOffset + ") startOffset is " +
|
firstChunkOffset + ") startOffset is " +
|
||||||
startOffset + " for file " + file);
|
params.getStartOffset() + " for file " + params.getFile());
|
||||||
}
|
}
|
||||||
|
|
||||||
return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
|
return new RemoteBlockReader2(params, checksum, firstChunkOffset, ch);
|
||||||
ch, checksum, verifyChecksum, startOffset, firstChunkOffset, len, sock,
|
|
||||||
ioStreams);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
static void checkSuccess(
|
static void checkSuccess(
|
||||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.BlockReader;
|
import org.apache.hadoop.hdfs.BlockReader;
|
||||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||||
|
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||||
import org.apache.hadoop.hdfs.DFSUtil;
|
import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
|
@ -201,14 +202,16 @@ public class JspHelper {
|
||||||
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||||
|
|
||||||
int amtToRead = (int)Math.min(chunkSizeToView, blockSize - offsetIntoBlock);
|
int amtToRead = (int)Math.min(chunkSizeToView, blockSize - offsetIntoBlock);
|
||||||
|
|
||||||
// Use the block name for file name.
|
// Use the block name for file name.
|
||||||
String file = BlockReaderFactory.getFileName(addr, poolId, blockId);
|
|
||||||
BlockReader blockReader = BlockReaderFactory.newBlockReader(
|
BlockReader blockReader = BlockReaderFactory.newBlockReader(
|
||||||
conf, s, file,
|
new BlockReaderFactory.Params(new Conf(conf)).
|
||||||
new ExtendedBlock(poolId, blockId, 0, genStamp), blockToken,
|
setSocket(s).
|
||||||
offsetIntoBlock, amtToRead, encryptionKey);
|
setBlockToken(blockToken).setStartOffset(offsetIntoBlock).
|
||||||
|
setLen(amtToRead).
|
||||||
|
setEncryptionKey(encryptionKey).
|
||||||
|
setFile(BlockReaderFactory.getFileName(addr, poolId, blockId)).
|
||||||
|
setBlock(new ExtendedBlock(poolId, blockId, 0, genStamp)));
|
||||||
byte[] buf = new byte[(int)amtToRead];
|
byte[] buf = new byte[(int)amtToRead];
|
||||||
int readOffset = 0;
|
int readOffset = 0;
|
||||||
int retries = 2;
|
int retries = 2;
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||||
import org.apache.hadoop.hdfs.BlockReader;
|
import org.apache.hadoop.hdfs.BlockReader;
|
||||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||||
import org.apache.hadoop.hdfs.DFSClient;
|
import org.apache.hadoop.hdfs.DFSClient;
|
||||||
|
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSUtil;
|
import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
|
@ -556,12 +557,14 @@ public class NamenodeFsck {
|
||||||
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
|
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
|
||||||
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||||
|
|
||||||
String file = BlockReaderFactory.getFileName(targetAddr, block.getBlockPoolId(),
|
|
||||||
block.getBlockId());
|
|
||||||
blockReader = BlockReaderFactory.newBlockReader(
|
blockReader = BlockReaderFactory.newBlockReader(
|
||||||
conf, s, file, block, lblock
|
new BlockReaderFactory.Params(new Conf(conf)).
|
||||||
.getBlockToken(), 0, -1,
|
setSocket(s).setBlock(block).
|
||||||
namenode.getRpcServer().getDataEncryptionKey());
|
setFile(BlockReaderFactory.getFileName(targetAddr,
|
||||||
|
block.getBlockPoolId(), block.getBlockId())).
|
||||||
|
setBlockToken(lblock.getBlockToken()).
|
||||||
|
setEncryptionKey(namenode.getRpcServer().getDataEncryptionKey()).
|
||||||
|
setLen(-1));
|
||||||
|
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
// Put chosen node into dead list, continue
|
// Put chosen node into dead list, continue
|
||||||
|
|
|
@ -31,6 +31,7 @@ import java.util.Random;
|
||||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
|
@ -150,12 +151,14 @@ public class BlockReaderTestUtil {
|
||||||
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||||
|
|
||||||
return BlockReaderFactory.newBlockReader(
|
return BlockReaderFactory.newBlockReader(
|
||||||
new DFSClient.Conf(conf),
|
new BlockReaderFactory.Params(new Conf(conf)).
|
||||||
sock, targetAddr.toString()+ ":" + block.getBlockId(), block,
|
setSocket(sock).
|
||||||
testBlock.getBlockToken(),
|
setFile(targetAddr.toString() + ":" + block.getBlockId()).
|
||||||
offset, lenToRead,
|
setBlock(block).setBlockToken(testBlock.getBlockToken()).
|
||||||
conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
|
setStartOffset(offset).setLen(lenToRead).
|
||||||
true, "", null, null);
|
setBufferSize(conf.getInt(
|
||||||
|
CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096)).
|
||||||
|
setVerifyChecksum(true));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.BlockReader;
|
import org.apache.hadoop.hdfs.BlockReader;
|
||||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||||
import org.apache.hadoop.hdfs.DFSClient;
|
import org.apache.hadoop.hdfs.DFSClient;
|
||||||
|
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
|
@ -145,9 +146,10 @@ public class TestBlockTokenWithDFS {
|
||||||
String file = BlockReaderFactory.getFileName(targetAddr,
|
String file = BlockReaderFactory.getFileName(targetAddr,
|
||||||
"test-blockpoolid", block.getBlockId());
|
"test-blockpoolid", block.getBlockId());
|
||||||
blockReader = BlockReaderFactory.newBlockReader(
|
blockReader = BlockReaderFactory.newBlockReader(
|
||||||
conf, s, file, block,
|
new BlockReaderFactory.Params(new Conf(conf)).
|
||||||
lblock.getBlockToken(), 0, -1, null);
|
setSocket(s).setBlock(block).setFile(file).
|
||||||
|
setBlockToken(lblock.getBlockToken()).setStartOffset(0).
|
||||||
|
setLen(-1));
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
if (ex instanceof InvalidBlockTokenException) {
|
if (ex instanceof InvalidBlockTokenException) {
|
||||||
assertFalse("OP_READ_BLOCK: access token is invalid, "
|
assertFalse("OP_READ_BLOCK: access token is invalid, "
|
||||||
|
|
|
@ -32,7 +32,9 @@ import java.util.Map;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hdfs.BlockReader;
|
||||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||||
|
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||||
|
@ -277,13 +279,13 @@ public class TestDataNodeVolumeFailure {
|
||||||
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
|
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
|
||||||
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||||
|
|
||||||
String file = BlockReaderFactory.getFileName(targetAddr,
|
BlockReader blockReader = BlockReaderFactory.
|
||||||
"test-blockpoolid",
|
newBlockReader(new BlockReaderFactory.Params(new Conf(conf)).
|
||||||
block.getBlockId());
|
setFile(BlockReaderFactory.getFileName(targetAddr,
|
||||||
BlockReaderFactory.newBlockReader(conf, s, file, block, lblock
|
"test-blockpoolid", block.getBlockId())).
|
||||||
.getBlockToken(), 0, -1, null);
|
setBlock(block).setBlockToken(lblock.getBlockToken()).
|
||||||
|
setSocket(s));
|
||||||
// nothing - if it fails - it will throw and exception
|
blockReader.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
Loading…
Reference in New Issue