|
@@ -30,7 +30,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
|
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
|
|
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
|
|
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
|
|
|
-import org.apache.hadoop.hdds.scm.storage.ChunkInputStream;
|
|
|
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
|
|
|
import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
|
|
|
import org.apache.ratis.util.Preconditions;
|
|
|
import org.slf4j.Logger;
|
|
@@ -44,17 +44,17 @@ import java.util.Arrays;
|
|
|
import java.util.List;
|
|
|
|
|
|
/**
|
|
|
- * Maintaining a list of ChunkInputStream. Read based on offset.
|
|
|
+ * Maintaining a list of BlockInputStream. Read based on offset.
|
|
|
*/
|
|
|
-public class ChunkGroupInputStream extends InputStream implements Seekable {
|
|
|
+public class KeyInputStream extends InputStream implements Seekable {
|
|
|
|
|
|
private static final Logger LOG =
|
|
|
- LoggerFactory.getLogger(ChunkGroupInputStream.class);
|
|
|
+ LoggerFactory.getLogger(KeyInputStream.class);
|
|
|
|
|
|
private static final int EOF = -1;
|
|
|
|
|
|
private final ArrayList<ChunkInputStreamEntry> streamEntries;
|
|
|
- // streamOffset[i] stores the offset at which chunkInputStream i stores
|
|
|
+ // streamOffset[i] stores the offset at which blockInputStream i stores
|
|
|
// data in the key
|
|
|
private long[] streamOffset = null;
|
|
|
private int currentStreamIndex;
|
|
@@ -62,7 +62,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
|
|
private boolean closed = false;
|
|
|
private String key;
|
|
|
|
|
|
- public ChunkGroupInputStream() {
|
|
|
+ public KeyInputStream() {
|
|
|
streamEntries = new ArrayList<>();
|
|
|
currentStreamIndex = 0;
|
|
|
}
|
|
@@ -84,7 +84,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
|
|
* @param streamLength the max number of bytes that should be written to this
|
|
|
* stream.
|
|
|
*/
|
|
|
- public synchronized void addStream(ChunkInputStream stream,
|
|
|
+ public synchronized void addStream(BlockInputStream stream,
|
|
|
long streamLength) {
|
|
|
streamEntries.add(new ChunkInputStreamEntry(stream, streamLength));
|
|
|
}
|
|
@@ -129,7 +129,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
|
|
// this case.
|
|
|
throw new IOException(String.format(
|
|
|
"Inconsistent read for blockID=%s length=%d numBytesRead=%d",
|
|
|
- current.chunkInputStream.getBlockID(), current.length,
|
|
|
+ current.blockInputStream.getBlockID(), current.length,
|
|
|
numBytesRead));
|
|
|
}
|
|
|
totalReadLen += numBytesRead;
|
|
@@ -174,7 +174,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
|
|
// accordingly so that currentStreamIndex = insertionPoint - 1
|
|
|
currentStreamIndex = -currentStreamIndex - 2;
|
|
|
}
|
|
|
- // seek to the proper offset in the ChunkInputStream
|
|
|
+ // seek to the proper offset in the BlockInputStream
|
|
|
streamEntries.get(currentStreamIndex)
|
|
|
.seek(pos - streamOffset[currentStreamIndex]);
|
|
|
}
|
|
@@ -207,17 +207,17 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Encapsulates ChunkInputStream.
|
|
|
+ * Encapsulates BlockInputStream.
|
|
|
*/
|
|
|
public static class ChunkInputStreamEntry extends InputStream
|
|
|
implements Seekable {
|
|
|
|
|
|
- private final ChunkInputStream chunkInputStream;
|
|
|
+ private final BlockInputStream blockInputStream;
|
|
|
private final long length;
|
|
|
|
|
|
- public ChunkInputStreamEntry(ChunkInputStream chunkInputStream,
|
|
|
+ public ChunkInputStreamEntry(BlockInputStream blockInputStream,
|
|
|
long length) {
|
|
|
- this.chunkInputStream = chunkInputStream;
|
|
|
+ this.blockInputStream = blockInputStream;
|
|
|
this.length = length;
|
|
|
}
|
|
|
|
|
@@ -228,29 +228,29 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
|
|
@Override
|
|
|
public synchronized int read(byte[] b, int off, int len)
|
|
|
throws IOException {
|
|
|
- int readLen = chunkInputStream.read(b, off, len);
|
|
|
+ int readLen = blockInputStream.read(b, off, len);
|
|
|
return readLen;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public synchronized int read() throws IOException {
|
|
|
- int data = chunkInputStream.read();
|
|
|
+ int data = blockInputStream.read();
|
|
|
return data;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public synchronized void close() throws IOException {
|
|
|
- chunkInputStream.close();
|
|
|
+ blockInputStream.close();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void seek(long pos) throws IOException {
|
|
|
- chunkInputStream.seek(pos);
|
|
|
+ blockInputStream.seek(pos);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public long getPos() throws IOException {
|
|
|
- return chunkInputStream.getPos();
|
|
|
+ return blockInputStream.getPos();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -267,7 +267,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
|
|
String requestId) throws IOException {
|
|
|
long length = 0;
|
|
|
long containerKey;
|
|
|
- ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream();
|
|
|
+ KeyInputStream groupInputStream = new KeyInputStream();
|
|
|
groupInputStream.key = keyInfo.getKeyName();
|
|
|
List<OmKeyLocationInfo> keyLocationInfos =
|
|
|
keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly();
|
|
@@ -304,7 +304,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
|
|
length += chunk.getLen();
|
|
|
}
|
|
|
success = true;
|
|
|
- ChunkInputStream inputStream = new ChunkInputStream(
|
|
|
+ BlockInputStream inputStream = new BlockInputStream(
|
|
|
omKeyLocationInfo.getBlockID(), xceiverClientManager, xceiverClient,
|
|
|
chunks, requestId);
|
|
|
groupInputStream.addStream(inputStream,
|