|
@@ -18,30 +18,18 @@
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import java.io.DataInputStream;
|
|
|
-import java.io.File;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import java.io.FileInputStream;
|
|
|
import java.io.IOException;
|
|
|
-import java.net.Socket;
|
|
|
import java.nio.ByteBuffer;
|
|
|
-import java.util.Collections;
|
|
|
-import java.util.HashMap;
|
|
|
-import java.util.LinkedHashMap;
|
|
|
-import java.util.Map;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
|
|
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
|
|
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
|
|
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
|
|
import org.apache.hadoop.hdfs.util.DirectBufferPool;
|
|
|
-import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
-import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
|
|
|
|
/**
|
|
@@ -53,74 +41,19 @@ import org.apache.hadoop.util.DataChecksum;
|
|
|
* <ul>
|
|
|
* <li>The client performing short circuit reads must be configured at the
|
|
|
* datanode.</li>
|
|
|
- * <li>The client gets the path to the file where block is stored using
|
|
|
- * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
|
|
|
- * RPC call</li>
|
|
|
- * <li>Client uses kerberos authentication to connect to the datanode over RPC,
|
|
|
- * if security is enabled.</li>
|
|
|
+ * <li>The client gets the file descriptors for the metadata file and the data
|
|
|
+ * file for the block using
|
|
|
+ * {@link org.apache.hadoop.hdfs.server.datanode.DataXceiver#requestShortCircuitFds}.
|
|
|
+ * </li>
|
|
|
+ * <li>The client reads the file descriptors.</li>
|
|
|
* </ul>
|
|
|
*/
|
|
|
class BlockReaderLocal implements BlockReader {
|
|
|
- private static final Log LOG = LogFactory.getLog(DFSClient.class);
|
|
|
-
|
|
|
- //Stores the cache and proxy for a local datanode.
|
|
|
- private static class LocalDatanodeInfo {
|
|
|
- private ClientDatanodeProtocol proxy = null;
|
|
|
- private final Map<ExtendedBlock, BlockLocalPathInfo> cache;
|
|
|
-
|
|
|
- LocalDatanodeInfo() {
|
|
|
- final int cacheSize = 10000;
|
|
|
- final float hashTableLoadFactor = 0.75f;
|
|
|
- int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
|
|
|
- cache = Collections
|
|
|
- .synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
|
|
|
- hashTableCapacity, hashTableLoadFactor, true) {
|
|
|
- private static final long serialVersionUID = 1;
|
|
|
-
|
|
|
- @Override
|
|
|
- protected boolean removeEldestEntry(
|
|
|
- Map.Entry<ExtendedBlock, BlockLocalPathInfo> eldest) {
|
|
|
- return size() > cacheSize;
|
|
|
- }
|
|
|
- });
|
|
|
- }
|
|
|
-
|
|
|
- private synchronized ClientDatanodeProtocol getDatanodeProxy(
|
|
|
- DatanodeInfo node, Configuration conf, int socketTimeout,
|
|
|
- boolean connectToDnViaHostname) throws IOException {
|
|
|
- if (proxy == null) {
|
|
|
- proxy = DFSUtil.createClientDatanodeProtocolProxy(node, conf,
|
|
|
- socketTimeout, connectToDnViaHostname);
|
|
|
- }
|
|
|
- return proxy;
|
|
|
- }
|
|
|
-
|
|
|
- private synchronized void resetDatanodeProxy() {
|
|
|
- if (null != proxy) {
|
|
|
- RPC.stopProxy(proxy);
|
|
|
- proxy = null;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) {
|
|
|
- return cache.get(b);
|
|
|
- }
|
|
|
-
|
|
|
- private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
|
|
|
- cache.put(b, info);
|
|
|
- }
|
|
|
-
|
|
|
- private void removeBlockLocalPathInfo(ExtendedBlock b) {
|
|
|
- cache.remove(b);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Multiple datanodes could be running on the local machine. Store proxies in
|
|
|
- // a map keyed by the ipc port of the datanode.
|
|
|
- private static Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
|
|
|
+ static final Log LOG = LogFactory.getLog(DFSClient.class);
|
|
|
|
|
|
private final FileInputStream dataIn; // reader for the data file
|
|
|
private final FileInputStream checksumIn; // reader for the checksum file
|
|
|
+ private final boolean verifyChecksum;
|
|
|
|
|
|
/**
|
|
|
* Offset from the most recent chunk boundary at which the next read should
|
|
@@ -140,7 +73,6 @@ class BlockReaderLocal implements BlockReader {
|
|
|
private ByteBuffer slowReadBuff = null;
|
|
|
private ByteBuffer checksumBuff = null;
|
|
|
private DataChecksum checksum;
|
|
|
- private final boolean verifyChecksum;
|
|
|
|
|
|
private static DirectBufferPool bufferPool = new DirectBufferPool();
|
|
|
|
|
@@ -150,186 +82,90 @@ class BlockReaderLocal implements BlockReader {
|
|
|
/** offset in block where reader wants to actually read */
|
|
|
private long startOffset;
|
|
|
private final String filename;
|
|
|
-
|
|
|
- /**
|
|
|
- * The only way this object can be instantiated.
|
|
|
- */
|
|
|
- static BlockReaderLocal newBlockReader(Configuration conf, String file,
|
|
|
- ExtendedBlock blk, Token<BlockTokenIdentifier> token, DatanodeInfo node,
|
|
|
- int socketTimeout, long startOffset, long length,
|
|
|
- boolean connectToDnViaHostname) throws IOException {
|
|
|
-
|
|
|
- LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
|
|
|
- .getIpcPort());
|
|
|
- // check the cache first
|
|
|
- BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
|
|
|
- if (pathinfo == null) {
|
|
|
- pathinfo = getBlockPathInfo(blk, node, conf, socketTimeout, token,
|
|
|
- connectToDnViaHostname);
|
|
|
- }
|
|
|
-
|
|
|
- // check to see if the file exists. It may so happen that the
|
|
|
- // HDFS file has been deleted and this block-lookup is occurring
|
|
|
- // on behalf of a new HDFS file. This time, the block file could
|
|
|
- // be residing in a different portion of the fs.data.dir directory.
|
|
|
- // In this case, we remove this entry from the cache. The next
|
|
|
- // call to this method will re-populate the cache.
|
|
|
- FileInputStream dataIn = null;
|
|
|
- FileInputStream checksumIn = null;
|
|
|
- BlockReaderLocal localBlockReader = null;
|
|
|
- boolean skipChecksumCheck = skipChecksumCheck(conf);
|
|
|
- try {
|
|
|
- // get a local file system
|
|
|
- File blkfile = new File(pathinfo.getBlockPath());
|
|
|
- dataIn = new FileInputStream(blkfile);
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("New BlockReaderLocal for file " + blkfile + " of size "
|
|
|
- + blkfile.length() + " startOffset " + startOffset + " length "
|
|
|
- + length + " short circuit checksum " + !skipChecksumCheck);
|
|
|
- }
|
|
|
|
|
|
- if (!skipChecksumCheck) {
|
|
|
- // get the metadata file
|
|
|
- File metafile = new File(pathinfo.getMetaPath());
|
|
|
- checksumIn = new FileInputStream(metafile);
|
|
|
-
|
|
|
- // read and handle the common header here. For now just a version
|
|
|
- BlockMetadataHeader header = BlockMetadataHeader
|
|
|
- .readHeader(new DataInputStream(checksumIn));
|
|
|
- short version = header.getVersion();
|
|
|
- if (version != BlockMetadataHeader.VERSION) {
|
|
|
- LOG.warn("Wrong version (" + version + ") for metadata file for "
|
|
|
- + blk + " ignoring ...");
|
|
|
- }
|
|
|
- DataChecksum checksum = header.getChecksum();
|
|
|
- long firstChunkOffset = startOffset
|
|
|
- - (startOffset % checksum.getBytesPerChecksum());
|
|
|
- localBlockReader = new BlockReaderLocal(conf, file, blk, token,
|
|
|
- startOffset, length, pathinfo, checksum, true, dataIn,
|
|
|
- firstChunkOffset, checksumIn);
|
|
|
- } else {
|
|
|
- localBlockReader = new BlockReaderLocal(conf, file, blk, token,
|
|
|
- startOffset, length, pathinfo, dataIn);
|
|
|
- }
|
|
|
- } catch (IOException e) {
|
|
|
- // remove from cache
|
|
|
- localDatanodeInfo.removeBlockLocalPathInfo(blk);
|
|
|
- DFSClient.LOG.warn("BlockReaderLocal: Removing " + blk
|
|
|
- + " from cache because local file " + pathinfo.getBlockPath()
|
|
|
- + " could not be opened.");
|
|
|
- throw e;
|
|
|
- } finally {
|
|
|
- if (localBlockReader == null) {
|
|
|
- if (dataIn != null) {
|
|
|
- dataIn.close();
|
|
|
- }
|
|
|
- if (checksumIn != null) {
|
|
|
- checksumIn.close();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- return localBlockReader;
|
|
|
- }
|
|
|
+ private final DatanodeID datanodeID;
|
|
|
+ private final ExtendedBlock block;
|
|
|
|
|
|
- private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
|
|
|
- LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
|
|
|
- if (ldInfo == null) {
|
|
|
- ldInfo = new LocalDatanodeInfo();
|
|
|
- localDatanodeInfoMap.put(port, ldInfo);
|
|
|
- }
|
|
|
- return ldInfo;
|
|
|
- }
|
|
|
-
|
|
|
- private static BlockLocalPathInfo getBlockPathInfo(ExtendedBlock blk,
|
|
|
- DatanodeInfo node, Configuration conf, int timeout,
|
|
|
- Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname)
|
|
|
- throws IOException {
|
|
|
- LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
|
|
|
- BlockLocalPathInfo pathinfo = null;
|
|
|
- ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(node,
|
|
|
- conf, timeout, connectToDnViaHostname);
|
|
|
- try {
|
|
|
- // make RPC to local datanode to find local pathnames of blocks
|
|
|
- pathinfo = proxy.getBlockLocalPathInfo(blk, token);
|
|
|
- if (pathinfo != null) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Cached location of block " + blk + " as " + pathinfo);
|
|
|
- }
|
|
|
- localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
|
|
|
- }
|
|
|
- } catch (IOException e) {
|
|
|
- localDatanodeInfo.resetDatanodeProxy(); // Reset proxy on error
|
|
|
- throw e;
|
|
|
- }
|
|
|
- return pathinfo;
|
|
|
- }
|
|
|
-
|
|
|
- private static boolean skipChecksumCheck(Configuration conf) {
|
|
|
- return conf.getBoolean(
|
|
|
- DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
|
|
- DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
|
|
|
- }
|
|
|
-
|
|
|
- private static int getSlowReadBufferNumChunks(Configuration conf, int bytesPerChecksum) {
|
|
|
- int bufferSizeBytes = conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
|
|
|
- DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
|
|
|
+ private static int getSlowReadBufferNumChunks(Configuration conf,
|
|
|
+ int bytesPerChecksum) {
|
|
|
|
|
|
- if (bufferSizeBytes < bytesPerChecksum) {
|
|
|
- throw new IllegalArgumentException("Configured BlockReaderLocal buffer size (" + bufferSizeBytes + ") " +
|
|
|
- "is not large enough to hold a single chunk (" + bytesPerChecksum + "). Please configure " +
|
|
|
+ int bufSize =
|
|
|
+ conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
|
|
|
+ DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
|
|
|
+
|
|
|
+ if (bufSize < bytesPerChecksum) {
|
|
|
+ throw new IllegalArgumentException("Configured BlockReaderLocal buffer size (" +
|
|
|
+ bufSize + ") is not large enough to hold a single chunk (" +
|
|
|
+ bytesPerChecksum + "). Please configure " +
|
|
|
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY + " appropriately");
|
|
|
}
|
|
|
|
|
|
// Round down to nearest chunk size
|
|
|
- return bufferSizeBytes / bytesPerChecksum;
|
|
|
+ return bufSize / bytesPerChecksum;
|
|
|
}
|
|
|
|
|
|
- private BlockReaderLocal(Configuration conf, String hdfsfile,
|
|
|
- ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
|
|
|
- long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn)
|
|
|
- throws IOException {
|
|
|
- this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
|
|
|
- DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
|
|
|
- dataIn, startOffset, null);
|
|
|
- }
|
|
|
-
|
|
|
- private BlockReaderLocal(Configuration conf, String hdfsfile,
|
|
|
- ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
|
|
|
- long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
|
|
|
- boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
|
|
|
- FileInputStream checksumIn) throws IOException {
|
|
|
- this.filename = hdfsfile;
|
|
|
- this.checksum = checksum;
|
|
|
- this.verifyChecksum = verifyChecksum;
|
|
|
- this.startOffset = Math.max(startOffset, 0);
|
|
|
-
|
|
|
- bytesPerChecksum = this.checksum.getBytesPerChecksum();
|
|
|
- checksumSize = this.checksum.getChecksumSize();
|
|
|
-
|
|
|
+ public BlockReaderLocal(Configuration conf, String filename,
|
|
|
+ ExtendedBlock block, long startOffset, long length,
|
|
|
+ FileInputStream dataIn, FileInputStream checksumIn,
|
|
|
+ DatanodeID datanodeID, boolean verifyChecksum) throws IOException {
|
|
|
this.dataIn = dataIn;
|
|
|
this.checksumIn = checksumIn;
|
|
|
- this.offsetFromChunkBoundary = (int) (startOffset-firstChunkOffset);
|
|
|
-
|
|
|
- int chunksPerChecksumRead = getSlowReadBufferNumChunks(conf, bytesPerChecksum);
|
|
|
- slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
|
|
|
- checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
|
|
|
- // Initially the buffers have nothing to read.
|
|
|
- slowReadBuff.flip();
|
|
|
- checksumBuff.flip();
|
|
|
+ this.startOffset = Math.max(startOffset, 0);
|
|
|
+ this.filename = filename;
|
|
|
+ this.datanodeID = datanodeID;
|
|
|
+ this.block = block;
|
|
|
+
|
|
|
+ // read and handle the common header here. For now just a version
|
|
|
+ checksumIn.getChannel().position(0);
|
|
|
+ BlockMetadataHeader header = BlockMetadataHeader
|
|
|
+ .readHeader(new DataInputStream(checksumIn));
|
|
|
+ short version = header.getVersion();
|
|
|
+ if (version != BlockMetadataHeader.VERSION) {
|
|
|
+ throw new IOException("Wrong version (" + version + ") of the " +
|
|
|
+ "metadata file for " + filename + ".");
|
|
|
+ }
|
|
|
+ if (!verifyChecksum) {
|
|
|
+ this.verifyChecksum = false;
|
|
|
+ } else {
|
|
|
+ this.verifyChecksum = !conf.getBoolean(DFSConfigKeys.
|
|
|
+ DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
|
|
+ DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
|
|
|
+ }
|
|
|
+ long firstChunkOffset;
|
|
|
+ if (this.verifyChecksum) {
|
|
|
+ this.checksum = header.getChecksum();
|
|
|
+ this.bytesPerChecksum = this.checksum.getBytesPerChecksum();
|
|
|
+ this.checksumSize = this.checksum.getChecksumSize();
|
|
|
+ firstChunkOffset = startOffset
|
|
|
+ - (startOffset % checksum.getBytesPerChecksum());
|
|
|
+ this.offsetFromChunkBoundary = (int) (startOffset - firstChunkOffset);
|
|
|
+
|
|
|
+ int chunksPerChecksumRead = getSlowReadBufferNumChunks(conf, bytesPerChecksum);
|
|
|
+ slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
|
|
|
+ checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
|
|
|
+ // Initially the buffers have nothing to read.
|
|
|
+ slowReadBuff.flip();
|
|
|
+ checksumBuff.flip();
|
|
|
+ long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
|
|
|
+ IOUtils.skipFully(checksumIn, checkSumOffset);
|
|
|
+ } else {
|
|
|
+ firstChunkOffset = startOffset;
|
|
|
+ this.checksum = null;
|
|
|
+ this.bytesPerChecksum = 0;
|
|
|
+ this.checksumSize = 0;
|
|
|
+ this.offsetFromChunkBoundary = 0;
|
|
|
+ }
|
|
|
+
|
|
|
boolean success = false;
|
|
|
try {
|
|
|
- // Skip both input streams to beginning of the chunk containing startOffset
|
|
|
- IOUtils.skipFully(dataIn, firstChunkOffset);
|
|
|
- if (checksumIn != null) {
|
|
|
- long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
|
|
|
- IOUtils.skipFully(checksumIn, checkSumOffset);
|
|
|
- }
|
|
|
+ // Reposition both input streams to the beginning of the chunk
|
|
|
+ // containing startOffset
|
|
|
+ this.dataIn.getChannel().position(firstChunkOffset);
|
|
|
success = true;
|
|
|
} finally {
|
|
|
if (!success) {
|
|
|
- bufferPool.returnBuffer(slowReadBuff);
|
|
|
- bufferPool.returnBuffer(checksumBuff);
|
|
|
+ if (slowReadBuff != null) bufferPool.returnBuffer(slowReadBuff);
|
|
|
+ if (checksumBuff != null) bufferPool.returnBuffer(checksumBuff);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -649,9 +485,17 @@ class BlockReaderLocal implements BlockReader {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized void close(PeerCache peerCache) throws IOException {
|
|
|
- dataIn.close();
|
|
|
- if (checksumIn != null) {
|
|
|
+ public synchronized void close(PeerCache peerCache,
|
|
|
+ FileInputStreamCache fisCache) throws IOException {
|
|
|
+ if (fisCache != null) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("putting FileInputStream for " + filename +
|
|
|
+ " back into FileInputStreamCache");
|
|
|
+ }
|
|
|
+ fisCache.put(datanodeID, block, new FileInputStream[] {dataIn, checksumIn});
|
|
|
+ } else {
|
|
|
+ LOG.debug("closing FileInputStream for " + filename);
|
|
|
+ dataIn.close();
|
|
|
checksumIn.close();
|
|
|
}
|
|
|
if (slowReadBuff != null) {
|
|
@@ -675,4 +519,10 @@ class BlockReaderLocal implements BlockReader {
|
|
|
public void readFully(byte[] buf, int off, int len) throws IOException {
|
|
|
BlockReaderUtil.readFully(this, buf, off, len);
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int available() throws IOException {
|
|
|
+ // We never do network I/O in BlockReaderLocal.
|
|
|
+ return Integer.MAX_VALUE;
|
|
|
+ }
|
|
|
}
|