|
@@ -19,10 +19,13 @@ package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import org.apache.hadoop.fs.ChecksumException;
|
|
|
+import org.apache.hadoop.fs.ReadOption;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.protocol.*;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
+import org.apache.hadoop.io.ByteBufferPool;
|
|
|
+
|
|
|
import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion;
|
|
|
import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions;
|
|
|
|
|
@@ -31,9 +34,11 @@ import org.apache.htrace.Span;
|
|
|
import org.apache.htrace.Trace;
|
|
|
import org.apache.htrace.TraceScope;
|
|
|
|
|
|
+import java.io.EOFException;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.nio.ByteBuffer;
|
|
|
+import java.util.EnumSet;
|
|
|
import java.util.Set;
|
|
|
import java.util.Map;
|
|
|
import java.util.HashMap;
|
|
@@ -263,6 +268,10 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
}
|
|
|
|
|
|
private long getOffsetInBlockGroup() {
|
|
|
+ return getOffsetInBlockGroup(pos);
|
|
|
+ }
|
|
|
+
|
|
|
+ private long getOffsetInBlockGroup(long pos) {
|
|
|
return pos - currentLocatedBlock.getStartOffset();
|
|
|
}
|
|
|
|
|
@@ -278,18 +287,22 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
// compute stripe range based on pos
|
|
|
final long offsetInBlockGroup = getOffsetInBlockGroup();
|
|
|
final long stripeLen = cellSize * dataBlkNum;
|
|
|
- int stripeIndex = (int) (offsetInBlockGroup / stripeLen);
|
|
|
- curStripeRange = new StripeRange(stripeIndex * stripeLen,
|
|
|
- Math.min(currentLocatedBlock.getBlockSize() - (stripeIndex * stripeLen),
|
|
|
- stripeLen));
|
|
|
- final int numCell = (int) ((curStripeRange.length - 1) / cellSize + 1);
|
|
|
+ final int stripeIndex = (int) (offsetInBlockGroup / stripeLen);
|
|
|
+ final int stripeBufOffset = (int) (offsetInBlockGroup % stripeLen);
|
|
|
+ final int stripeLimit = (int) Math.min(currentLocatedBlock.getBlockSize()
|
|
|
+ - (stripeIndex * stripeLen), stripeLen);
|
|
|
+ curStripeRange = new StripeRange(offsetInBlockGroup,
|
|
|
+ stripeLimit - stripeBufOffset);
|
|
|
+
|
|
|
+ final int startCell = stripeBufOffset / cellSize;
|
|
|
+ final int numCell = (stripeLimit - 1) / cellSize + 1;
|
|
|
|
|
|
// read the whole stripe in parallel
|
|
|
Map<Future<Integer>, Integer> futures = new HashMap<>();
|
|
|
- for (int i = 0; i < numCell; i++) {
|
|
|
- curStripeBuf.position(cellSize * i);
|
|
|
- curStripeBuf.limit((int) Math.min(cellSize * (i + 1),
|
|
|
- curStripeRange.length));
|
|
|
+ for (int i = startCell; i < numCell; i++) {
|
|
|
+ int bufPos = i == startCell ? stripeBufOffset : cellSize * i;
|
|
|
+ curStripeBuf.position(bufPos);
|
|
|
+ curStripeBuf.limit(Math.min(cellSize * (i + 1), stripeLimit));
|
|
|
ByteBuffer buf = curStripeBuf.slice();
|
|
|
ByteBufferStrategy strategy = new ByteBufferStrategy(buf);
|
|
|
final int targetLength = buf.remaining();
|
|
@@ -329,6 +342,39 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
};
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Seek to a new arbitrary location
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public synchronized void seek(long targetPos) throws IOException {
|
|
|
+ if (targetPos > getFileLength()) {
|
|
|
+ throw new EOFException("Cannot seek after EOF");
|
|
|
+ }
|
|
|
+ if (targetPos < 0) {
|
|
|
+ throw new EOFException("Cannot seek to negative offset");
|
|
|
+ }
|
|
|
+ if (closed.get()) {
|
|
|
+ throw new IOException("Stream is closed!");
|
|
|
+ }
|
|
|
+ if (targetPos <= blockEnd) {
|
|
|
+ final long targetOffsetInBlk = getOffsetInBlockGroup(targetPos);
|
|
|
+ if (curStripeRange.include(targetOffsetInBlk)) {
|
|
|
+ int bufOffset = getStripedBufOffset(targetOffsetInBlk);
|
|
|
+ curStripeBuf.position(bufOffset);
|
|
|
+ pos = targetPos;
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ pos = targetPos;
|
|
|
+ blockEnd = -1;
|
|
|
+ }
|
|
|
+
|
|
|
+ private int getStripedBufOffset(long offsetInBlockGroup) {
|
|
|
+ final long stripeLen = cellSize * dataBlkNum;
|
|
|
+ // compute the position in the curStripeBuf based on "pos"
|
|
|
+ return (int) (offsetInBlockGroup % stripeLen);
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
protected synchronized int readWithStrategy(ReaderStrategy strategy,
|
|
|
int off, int len) throws IOException {
|
|
@@ -405,10 +451,8 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
* @return number of bytes copied
|
|
|
*/
|
|
|
private int copy(ReaderStrategy strategy, int offset, int length) {
|
|
|
- final long stripeLen = cellSize * dataBlkNum;
|
|
|
- final long offsetInBlk = pos - currentLocatedBlock.getStartOffset();
|
|
|
- // compute the position in the curStripeBuf based on "pos"
|
|
|
- int bufOffset = (int) (offsetInBlk % stripeLen);
|
|
|
+ final long offsetInBlk = getOffsetInBlockGroup();
|
|
|
+ int bufOffset = getStripedBufOffset(offsetInBlk);
|
|
|
curStripeBuf.position(bufOffset);
|
|
|
return strategy.copyFrom(curStripeBuf, offset,
|
|
|
Math.min(length, curStripeBuf.remaining()));
|
|
@@ -546,4 +590,22 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
}
|
|
|
throw new InterruptedException("let's retry");
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * May need online read recovery, zero-copy read doesn't make
|
|
|
+ * sense, so don't support it.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public synchronized ByteBuffer read(ByteBufferPool bufferPool,
|
|
|
+ int maxLength, EnumSet<ReadOption> opts)
|
|
|
+ throws IOException, UnsupportedOperationException {
|
|
|
+ throw new UnsupportedOperationException(
|
|
|
+ "Not support enhanced byte buffer access.");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void releaseBuffer(ByteBuffer buffer) {
|
|
|
+ throw new UnsupportedOperationException(
|
|
|
+ "Not support enhanced byte buffer access.");
|
|
|
+ }
|
|
|
}
|