|
@@ -31,7 +31,6 @@ import com.google.protobuf.ByteString;
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ChunkInfo;
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.KeyData;
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.KeyValue;
|
|
|
-import org.apache.hadoop.scm.ScmConfigKeys;
|
|
|
import org.apache.hadoop.scm.XceiverClientManager;
|
|
|
import org.apache.hadoop.scm.XceiverClientSpi;
|
|
|
|
|
@@ -62,6 +61,7 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
private ByteBuffer buffer;
|
|
|
private final String streamId;
|
|
|
private int chunkIndex;
|
|
|
+ private int chunkSize;
|
|
|
|
|
|
/**
|
|
|
* Creates a new ChunkOutputStream.
|
|
@@ -71,13 +71,15 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
* @param xceiverClientManager client manager that controls client
|
|
|
* @param xceiverClient client to perform container calls
|
|
|
* @param traceID container protocol call args
|
|
|
+ * @param chunkSize chunk size
|
|
|
*/
|
|
|
public ChunkOutputStream(String containerKey, String key,
|
|
|
XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
|
|
|
- String traceID) {
|
|
|
+ String traceID, int chunkSize) {
|
|
|
this.containerKey = containerKey;
|
|
|
this.key = key;
|
|
|
this.traceID = traceID;
|
|
|
+ this.chunkSize = chunkSize;
|
|
|
KeyValue keyValue = KeyValue.newBuilder()
|
|
|
.setKey("TYPE").setValue("KEY").build();
|
|
|
this.containerKeyData = KeyData.newBuilder()
|
|
@@ -86,7 +88,7 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
.addMetadata(keyValue);
|
|
|
this.xceiverClientManager = xceiverClientManager;
|
|
|
this.xceiverClient = xceiverClient;
|
|
|
- this.buffer = ByteBuffer.allocate(ScmConfigKeys.CHUNK_SIZE);
|
|
|
+ this.buffer = ByteBuffer.allocate(chunkSize);
|
|
|
this.streamId = UUID.randomUUID().toString();
|
|
|
this.chunkIndex = 0;
|
|
|
}
|
|
@@ -97,7 +99,7 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
int rollbackPosition = buffer.position();
|
|
|
int rollbackLimit = buffer.limit();
|
|
|
buffer.put((byte)b);
|
|
|
- if (buffer.position() == ScmConfigKeys.CHUNK_SIZE) {
|
|
|
+ if (buffer.position() == chunkSize) {
|
|
|
flushBufferToChunk(rollbackPosition, rollbackLimit);
|
|
|
}
|
|
|
}
|
|
@@ -116,12 +118,11 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
}
|
|
|
checkOpen();
|
|
|
while (len > 0) {
|
|
|
- int writeLen = Math.min(
|
|
|
- ScmConfigKeys.CHUNK_SIZE - buffer.position(), len);
|
|
|
+ int writeLen = Math.min(chunkSize - buffer.position(), len);
|
|
|
int rollbackPosition = buffer.position();
|
|
|
int rollbackLimit = buffer.limit();
|
|
|
buffer.put(b, off, writeLen);
|
|
|
- if (buffer.position() == ScmConfigKeys.CHUNK_SIZE) {
|
|
|
+ if (buffer.position() == chunkSize) {
|
|
|
flushBufferToChunk(rollbackPosition, rollbackLimit);
|
|
|
}
|
|
|
off += writeLen;
|