|
@@ -29,7 +29,6 @@ import org.apache.hadoop.ozone.common.Checksum;
|
|
|
import org.apache.hadoop.ozone.common.ChecksumData;
|
|
|
import org.apache.hadoop.ozone.common.OzoneChecksumException;
|
|
|
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
|
|
|
-import org.apache.commons.codec.digest.DigestUtils;
|
|
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
|
|
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
|
|
@@ -44,7 +43,6 @@ import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.Collections;
|
|
|
-import java.util.UUID;
|
|
|
import java.util.List;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Map;
|
|
@@ -81,14 +79,12 @@ public class BlockOutputStream extends OutputStream {
|
|
|
LoggerFactory.getLogger(BlockOutputStream.class);
|
|
|
|
|
|
private volatile BlockID blockID;
|
|
|
- private final String key;
|
|
|
|
|
|
private final BlockData.Builder containerBlockData;
|
|
|
private XceiverClientManager xceiverClientManager;
|
|
|
private XceiverClientSpi xceiverClient;
|
|
|
private final ContainerProtos.ChecksumType checksumType;
|
|
|
private final int bytesPerChecksum;
|
|
|
- private final String streamId;
|
|
|
private int chunkIndex;
|
|
|
private int chunkSize;
|
|
|
private final long streamBufferFlushSize;
|
|
@@ -125,7 +121,6 @@ public class BlockOutputStream extends OutputStream {
|
|
|
* Creates a new BlockOutputStream.
|
|
|
*
|
|
|
* @param blockID block ID
|
|
|
- * @param key chunk key
|
|
|
* @param xceiverClientManager client manager that controls client
|
|
|
* @param pipeline pipeline where block will be written
|
|
|
* @param chunkSize chunk size
|
|
@@ -137,14 +132,13 @@ public class BlockOutputStream extends OutputStream {
|
|
|
* @param bytesPerChecksum Bytes per checksum
|
|
|
*/
|
|
|
@SuppressWarnings("parameternumber")
|
|
|
- public BlockOutputStream(BlockID blockID, String key,
|
|
|
+ public BlockOutputStream(BlockID blockID,
|
|
|
XceiverClientManager xceiverClientManager, Pipeline pipeline,
|
|
|
- int chunkSize, long streamBufferFlushSize,
|
|
|
- long streamBufferMaxSize, long watchTimeout, BufferPool bufferPool,
|
|
|
- ChecksumType checksumType, int bytesPerChecksum)
|
|
|
+ int chunkSize, long streamBufferFlushSize, long streamBufferMaxSize,
|
|
|
+ long watchTimeout, BufferPool bufferPool, ChecksumType checksumType,
|
|
|
+ int bytesPerChecksum)
|
|
|
throws IOException {
|
|
|
this.blockID = blockID;
|
|
|
- this.key = key;
|
|
|
this.chunkSize = chunkSize;
|
|
|
KeyValue keyValue =
|
|
|
KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
|
|
@@ -153,7 +147,6 @@ public class BlockOutputStream extends OutputStream {
|
|
|
.addMetadata(keyValue);
|
|
|
this.xceiverClientManager = xceiverClientManager;
|
|
|
this.xceiverClient = xceiverClientManager.acquireClient(pipeline);
|
|
|
- this.streamId = UUID.randomUUID().toString();
|
|
|
this.chunkIndex = 0;
|
|
|
this.streamBufferFlushSize = streamBufferFlushSize;
|
|
|
this.streamBufferMaxSize = streamBufferMaxSize;
|
|
@@ -598,8 +591,7 @@ public class BlockOutputStream extends OutputStream {
|
|
|
Checksum checksum = new Checksum(checksumType, bytesPerChecksum);
|
|
|
ChecksumData checksumData = checksum.computeChecksum(chunk);
|
|
|
ChunkInfo chunkInfo = ChunkInfo.newBuilder()
|
|
|
- .setChunkName(DigestUtils.md5Hex(key) + "_stream_" + streamId +
|
|
|
- "_chunk_" + ++chunkIndex)
|
|
|
+ .setChunkName(blockID.getLocalID() + "_chunk_" + ++chunkIndex)
|
|
|
.setOffset(0)
|
|
|
.setLen(effectiveChunkSize)
|
|
|
.setChecksumData(checksumData.getProtoBufMessage())
|