|
@@ -18,20 +18,21 @@
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
+import org.apache.hadoop.fs.ChecksumException;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
-import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
|
-import org.apache.hadoop.hdfs.protocol.ECInfo;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
|
|
|
+import org.apache.hadoop.hdfs.protocol.*;
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
|
|
|
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
+import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.htrace.Span;
|
|
|
import org.apache.htrace.Trace;
|
|
|
import org.apache.htrace.TraceScope;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.net.InetSocketAddress;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
@@ -125,6 +126,9 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
return results;
|
|
|
}
|
|
|
|
|
|
+ private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS;
|
|
|
+ private BlockReader[] blockReaders = null;
|
|
|
+ private DatanodeInfo[] currentNodes = null;
|
|
|
private final int cellSize;
|
|
|
private final short dataBlkNum;
|
|
|
private final short parityBlkNum;
|
|
@@ -143,13 +147,285 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
|
|
|
@Override
|
|
|
public synchronized int read(final ByteBuffer buf) throws IOException {
|
|
|
- throw new UnsupportedActionException("Stateful read is not supported");
|
|
|
+ ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
|
|
|
+ TraceScope scope =
|
|
|
+ dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src);
|
|
|
+ try {
|
|
|
+ return readWithStrategy(byteBufferReader, 0, buf.remaining());
|
|
|
+ } finally {
|
|
|
+ scope.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * When seeking into a new block group, create blockReader for each internal
|
|
|
+ * block in the group.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ private synchronized DatanodeInfo[] blockSeekTo(long target)
|
|
|
+ throws IOException {
|
|
|
+ if (target >= getFileLength()) {
|
|
|
+ throw new IOException("Attempted to read past end of file");
|
|
|
+ }
|
|
|
+
|
|
|
+ // Will be getting a new BlockReader.
|
|
|
+ closeCurrentBlockReaders();
|
|
|
+
|
|
|
+ // Connect to best DataNode for desired Block, with potential offset
|
|
|
+ DatanodeInfo[] chosenNodes = new DatanodeInfo[groupSize];
|
|
|
+ int refetchToken = 1; // only need to get a new access token once
|
|
|
+ int refetchEncryptionKey = 1; // only need to get a new encryption key once
|
|
|
+
|
|
|
+ // Compute desired striped block group
|
|
|
+ LocatedStripedBlock targetBlockGroup = getBlockGroupAt(target);
|
|
|
+
|
|
|
+ // Update current position
|
|
|
+ this.pos = target;
|
|
|
+ this.blockEnd = targetBlockGroup.getStartOffset() +
|
|
|
+ targetBlockGroup.getBlockSize() - 1;
|
|
|
+
|
|
|
+ long offsetIntoBlockGroup = target - targetBlockGroup.getStartOffset();
|
|
|
+ LocatedBlock[] targetBlocks = StripedBlockUtil.parseStripedBlockGroup(
|
|
|
+ targetBlockGroup, cellSize, dataBlkNum, parityBlkNum);
|
|
|
+ // The purpose is to get start offset into each block
|
|
|
+ ReadPortion[] readPortions = planReadPortions(groupSize, cellSize,
|
|
|
+ offsetIntoBlockGroup, 0, 0);
|
|
|
+ while (true) {
|
|
|
+ int i = 0;
|
|
|
+ InetSocketAddress targetAddr = null;
|
|
|
+ try {
|
|
|
+ blockReaders = new BlockReader[groupSize];
|
|
|
+ for (i = 0; i < groupSize; i++) {
|
|
|
+ LocatedBlock targetBlock = targetBlocks[i];
|
|
|
+ if (targetBlock == null) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ long offsetIntoBlock = readPortions[i].startOffsetInBlock;
|
|
|
+ DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null);
|
|
|
+ chosenNodes[i] = retval.info;
|
|
|
+ targetAddr = retval.addr;
|
|
|
+ StorageType storageType = retval.storageType;
|
|
|
+
|
|
|
+ ExtendedBlock blk = targetBlock.getBlock();
|
|
|
+ Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
|
|
|
+ CachingStrategy curCachingStrategy;
|
|
|
+ boolean shortCircuitForbidden;
|
|
|
+ synchronized(infoLock) {
|
|
|
+ curCachingStrategy = cachingStrategy;
|
|
|
+ shortCircuitForbidden = shortCircuitForbidden();
|
|
|
+ }
|
|
|
+ blockReaders[i] = new BlockReaderFactory(dfsClient.getConf()).
|
|
|
+ setInetSocketAddress(targetAddr).
|
|
|
+ setRemotePeerFactory(dfsClient).
|
|
|
+ setDatanodeInfo(chosenNodes[i]).
|
|
|
+ setStorageType(storageType).
|
|
|
+ setFileName(src).
|
|
|
+ setBlock(blk).
|
|
|
+ setBlockToken(accessToken).
|
|
|
+ setStartOffset(offsetIntoBlock).
|
|
|
+ setVerifyChecksum(verifyChecksum).
|
|
|
+ setClientName(dfsClient.clientName).
|
|
|
+ setLength(blk.getNumBytes() - offsetIntoBlock).
|
|
|
+ setCachingStrategy(curCachingStrategy).
|
|
|
+ setAllowShortCircuitLocalReads(!shortCircuitForbidden).
|
|
|
+ setClientCacheContext(dfsClient.getClientContext()).
|
|
|
+ setUserGroupInformation(dfsClient.ugi).
|
|
|
+ setConfiguration(dfsClient.getConfiguration()).
|
|
|
+ build();
|
|
|
+ }
|
|
|
+ currentLocatedBlock = targetBlockGroup;
|
|
|
+ return chosenNodes;
|
|
|
+ } catch (IOException ex) {
|
|
|
+ // Retry in case of encryption key or token exceptions. Otherwise throw
|
|
|
+ // IOException: since each internal block is singly replicated, it's
|
|
|
+ // not meaningful trying to locate another replica.
|
|
|
+ if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
|
|
+ DFSClient.LOG.info("Will fetch a new encryption key and retry, "
|
|
|
+ + "encryption key was invalid when connecting to " + targetAddr
|
|
|
+ + " : " + ex);
|
|
|
+ // The encryption key used is invalid.
|
|
|
+ refetchEncryptionKey--;
|
|
|
+ dfsClient.clearDataEncryptionKey();
|
|
|
+ } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
|
|
|
+ refetchToken--;
|
|
|
+ fetchBlockAt(target);
|
|
|
+ } else {
|
|
|
+ DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
|
|
|
+ + ", add to deadNodes and continue. " + ex, ex);
|
|
|
+ // Put chosen node into dead list and throw exception
|
|
|
+ addToDeadNodes(chosenNodes[i]);
|
|
|
+ throw ex;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Extend the super method with the logic of switching between cells.
|
|
|
+ * When reaching the end of a cell, proceed to the next cell and read it
|
|
|
+ * with the next blockReader.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ protected void closeCurrentBlockReaders() {
|
|
|
+ if (blockReaders == null || blockReaders.length == 0) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ for (int i = 0; i < groupSize; i++) {
|
|
|
+ if (blockReaders[i] == null) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ blockReaders[i].close();
|
|
|
+ } catch (IOException e) {
|
|
|
+ DFSClient.LOG.error("error closing blockReader", e);
|
|
|
+ }
|
|
|
+ blockReaders[i] = null;
|
|
|
+ }
|
|
|
+ blockEnd = -1;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized int read(final byte buf[], int off, int len)
|
|
|
+ protected synchronized int readWithStrategy(ReaderStrategy strategy,
|
|
|
+ int off, int len) throws IOException {
|
|
|
+ dfsClient.checkOpen();
|
|
|
+ if (closed.get()) {
|
|
|
+ throw new IOException("Stream closed");
|
|
|
+ }
|
|
|
+ Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
|
|
|
+ = new HashMap<>();
|
|
|
+ failures = 0;
|
|
|
+ if (pos < getFileLength()) {
|
|
|
+ int retries = 2;
|
|
|
+ /** Index of the target block in a stripe to read from */
|
|
|
+ int idxInGroup = (int) ((pos / cellSize) % dataBlkNum);
|
|
|
+ while (retries > 0) {
|
|
|
+ try {
|
|
|
+ // currentNode can be left as null if previous read had a checksum
|
|
|
+ // error on the same block. See HDFS-3067
|
|
|
+ if (pos > blockEnd || currentNodes == null) {
|
|
|
+ currentNodes = blockSeekTo(pos);
|
|
|
+ }
|
|
|
+ int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
|
|
|
+ synchronized(infoLock) {
|
|
|
+ if (locatedBlocks.isLastBlockComplete()) {
|
|
|
+ realLen = (int) Math.min(realLen,
|
|
|
+ locatedBlocks.getFileLength() - pos);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Number of bytes already read into buffer */
|
|
|
+ int result = 0;
|
|
|
+ while (result < realLen) {
|
|
|
+ /**
|
|
|
+ * Temporary position into the file; {@link pos} might not proceed
|
|
|
+ * to this temporary position in case of exceptions.
|
|
|
+ */
|
|
|
+ long tmpPos = pos + result;
|
|
|
+ /** Start and end offsets of a cell in the file */
|
|
|
+ long cellStart = (tmpPos / cellSize) * cellSize;
|
|
|
+ long cellEnd = cellStart + cellSize - 1;
|
|
|
+
|
|
|
+ /** Number of bytes to read from the current cell */
|
|
|
+ int realLenInCell = (int) Math.min(realLen - result,
|
|
|
+ cellEnd - tmpPos + 1L);
|
|
|
+ assert realLenInCell > 0 : "Temporary position shouldn't be " +
|
|
|
+ "after cellEnd";
|
|
|
+ // Read from one blockReader up to cell boundary
|
|
|
+ int cellRet = readBuffer(blockReaders[idxInGroup],
|
|
|
+ currentNodes[idxInGroup], strategy, off + result,
|
|
|
+ realLenInCell);
|
|
|
+ if (cellRet >= 0) {
|
|
|
+ result += cellRet;
|
|
|
+ if (cellRet < realLenInCell) {
|
|
|
+ // A short read indicates the current blockReader buffer is
|
|
|
+ // already drained. Should return the read call. Otherwise
|
|
|
+ // should proceed to the next cell.
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ // got a EOS from reader though we expect more data on it.
|
|
|
+ throw new IOException("Unexpected EOS from the reader");
|
|
|
+ }
|
|
|
+ idxInGroup = (idxInGroup + 1) % dataBlkNum;
|
|
|
+ }
|
|
|
+
|
|
|
+ pos += result;
|
|
|
+
|
|
|
+ if (dfsClient.stats != null) {
|
|
|
+ dfsClient.stats.incrementBytesRead(result);
|
|
|
+ }
|
|
|
+ return result;
|
|
|
+ } catch (ChecksumException ce) {
|
|
|
+ throw ce;
|
|
|
+ } catch (IOException e) {
|
|
|
+ if (retries == 1) {
|
|
|
+ DFSClient.LOG.warn("DFS Read", e);
|
|
|
+ }
|
|
|
+ blockEnd = -1;
|
|
|
+ if (currentNodes[idxInGroup] != null) {
|
|
|
+ addToDeadNodes(currentNodes[idxInGroup]);
|
|
|
+ }
|
|
|
+ if (--retries == 0) {
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ // Check if need to report block replicas corruption either read
|
|
|
+ // was successful or ChecksumException occured.
|
|
|
+ reportCheckSumFailure(corruptedBlockMap,
|
|
|
+ currentLocatedBlock.getLocations().length);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return -1;
|
|
|
+ }
|
|
|
+
|
|
|
+ private synchronized int readBuffer(BlockReader blockReader,
|
|
|
+ DatanodeInfo currentNode, ReaderStrategy readerStrategy, int off, int len)
|
|
|
+ throws IOException {
|
|
|
+ IOException ioe;
|
|
|
+ while (true) {
|
|
|
+ try {
|
|
|
+ return readerStrategy.doRead(blockReader, off, len);
|
|
|
+ } catch ( ChecksumException ce ) {
|
|
|
+ DFSClient.LOG.warn("Found Checksum error for "
|
|
|
+ + getCurrentBlock() + " from " + currentNode
|
|
|
+ + " at " + ce.getPos());
|
|
|
+ // If current block group is corrupt, it's meaningless to retry.
|
|
|
+ // TODO: this should trigger decoding logic (HDFS-7678)
|
|
|
+ throw ce;
|
|
|
+ } catch ( IOException e ) {
|
|
|
+ ioe = e;
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean sourceFound = seekToBlockSource(pos);
|
|
|
+ if (!sourceFound) {
|
|
|
+ throw ioe;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean seekToBlockSource(long targetPos)
|
|
|
throws IOException {
|
|
|
- throw new UnsupportedActionException("Stateful read is not supported");
|
|
|
+ currentNodes = blockSeekTo(targetPos);
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ protected class ByteBufferStrategy extends DFSInputStream.ByteBufferStrategy {
|
|
|
+ ByteBufferStrategy(ByteBuffer buf) {
|
|
|
+ super(buf);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int doRead(BlockReader blockReader, int off, int len)
|
|
|
+ throws ChecksumException, IOException {
|
|
|
+ int oldlimit = buf.limit();
|
|
|
+ if (buf.remaining() > len) {
|
|
|
+ buf.limit(buf.position() + len);
|
|
|
+ }
|
|
|
+ int ret = super.doRead(blockReader, off, len);
|
|
|
+ buf.limit(oldlimit);
|
|
|
+ return ret;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -188,8 +464,11 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
dataBlkNum, idx);
|
|
|
}
|
|
|
|
|
|
- private LocatedBlock getBlockGroupAt(long offset) throws IOException {
|
|
|
- return super.getBlockAt(offset);
|
|
|
+ private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException {
|
|
|
+ LocatedBlock lb = super.getBlockAt(offset);
|
|
|
+ assert lb instanceof LocatedStripedBlock : "NameNode" +
|
|
|
+ " should return a LocatedStripedBlock for a striped file";
|
|
|
+ return (LocatedStripedBlock)lb;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -206,10 +485,8 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
int len = (int) (end - start + 1);
|
|
|
|
|
|
// Refresh the striped block group
|
|
|
- LocatedBlock block = getBlockGroupAt(blockStartOffset);
|
|
|
- assert block instanceof LocatedStripedBlock : "NameNode" +
|
|
|
- " should return a LocatedStripedBlock for a striped file";
|
|
|
- LocatedStripedBlock blockGroup = (LocatedStripedBlock) block;
|
|
|
+ LocatedStripedBlock blockGroup = getBlockGroupAt(blockStartOffset);
|
|
|
+
|
|
|
|
|
|
// Planning the portion of I/O for each shard
|
|
|
ReadPortion[] readPortions = planReadPortions(dataBlkNum, cellSize, start,
|
|
@@ -308,7 +585,7 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
* +------------------------------------------------------+
|
|
|
*/
|
|
|
private long startOffsetInBlock = 0;
|
|
|
- private long readLength = 0;
|
|
|
+ private int readLength = 0;
|
|
|
private final List<Integer> offsetsInBuf = new ArrayList<>();
|
|
|
private final List<Integer> lengths = new ArrayList<>();
|
|
|
|
|
@@ -328,7 +605,7 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
return lens;
|
|
|
}
|
|
|
|
|
|
- long getReadLength() {
|
|
|
+ int getReadLength() {
|
|
|
return readLength;
|
|
|
}
|
|
|
|