Browse Source

HDDS-1492. Generated chunk size name too long. Contributed by Shashikannt Banerjee. (#1084)

Shashikant Banerjee 5 years ago
parent
commit
d21eccf8ba

+ 5 - 13
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java

@@ -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())

+ 1 - 1
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java

@@ -108,7 +108,7 @@ public final class BlockOutputStreamEntry extends OutputStream {
         UserGroupInformation.getCurrentUser().addToken(getToken());
       }
       this.outputStream =
-          new BlockOutputStream(blockID, key, xceiverClientManager,
+          new BlockOutputStream(blockID, xceiverClientManager,
               pipeline, chunkSize, streamBufferFlushSize,
               streamBufferMaxSize, watchTimeout, bufferPool, checksumType,
               bytesPerChecksum);