You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/02/16 02:59:29 UTC

[GitHub] [ozone] rakeshadr opened a new pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

rakeshadr opened a new pull request #1923:
URL: https://github.com/apache/ozone/pull/1923


   ## What changes were proposed in this pull request?
   
   This task is to implement prefix based FSO for the S3MultipartUploadCompleteRequest. 
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-4813
   
   ## How was this patch tested?
   
   Added UT cases
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] linyiqun commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r576591831



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadV1.java
##########
@@ -264,12 +274,232 @@ public void testUploadPartOverrideWithRatis() throws IOException {
     commitUploadPartInfo = ozoneOutputStream
             .getCommitUploadPartInfo();
 
-    assertNotNull(commitUploadPartInfo);
-    assertNotNull(commitUploadPartInfo.getPartName());
+    Assert.assertNotNull(commitUploadPartInfo);
+    Assert.assertNotNull(commitUploadPartInfo.getPartName());
 
     // PartName should be different from old part Name.
-    assertNotEquals("Part names should be different", partName,
+    Assert.assertNotEquals("Part names should be different", partName,
             commitUploadPartInfo.getPartName());
   }
 
+  @Test
+  public void testMultipartUploadWithPartsLessThanMinSize() throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    // Initiate multipart upload
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    // Upload Parts
+    Map<Integer, String> partsMap = new TreeMap<>();
+    // Uploading part 1 with less than min size
+    String partName = uploadPart(bucket, keyName, uploadID, 1,
+            "data".getBytes(UTF_8));
+    partsMap.put(1, partName);
+
+    partName = uploadPart(bucket, keyName, uploadID, 2,
+            "data".getBytes(UTF_8));
+    partsMap.put(2, partName);
+
+    // Complete multipart upload
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.ENTITY_TOO_SMALL,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithPartsMisMatchWithListSizeDifferent()
+          throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(1, UUID.randomUUID().toString());
+
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.INVALID_PART,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithPartsMisMatchWithIncorrectPartName()
+          throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(1, UUID.randomUUID().toString());

Review comment:
       The comment should be updated, this case is passing with an incorrect part name.

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadV1.java
##########
@@ -264,12 +274,232 @@ public void testUploadPartOverrideWithRatis() throws IOException {
     commitUploadPartInfo = ozoneOutputStream
             .getCommitUploadPartInfo();
 
-    assertNotNull(commitUploadPartInfo);
-    assertNotNull(commitUploadPartInfo.getPartName());
+    Assert.assertNotNull(commitUploadPartInfo);
+    Assert.assertNotNull(commitUploadPartInfo.getPartName());
 
     // PartName should be different from old part Name.
-    assertNotEquals("Part names should be different", partName,
+    Assert.assertNotEquals("Part names should be different", partName,
             commitUploadPartInfo.getPartName());
   }
 
+  @Test
+  public void testMultipartUploadWithPartsLessThanMinSize() throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    // Initiate multipart upload
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    // Upload Parts
+    Map<Integer, String> partsMap = new TreeMap<>();
+    // Uploading part 1 with less than min size
+    String partName = uploadPart(bucket, keyName, uploadID, 1,
+            "data".getBytes(UTF_8));
+    partsMap.put(1, partName);
+
+    partName = uploadPart(bucket, keyName, uploadID, 2,
+            "data".getBytes(UTF_8));
+    partsMap.put(2, partName);
+
+    // Complete multipart upload
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.ENTITY_TOO_SMALL,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithPartsMisMatchWithListSizeDifferent()
+          throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(1, UUID.randomUUID().toString());
+
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.INVALID_PART,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithPartsMisMatchWithIncorrectPartName()
+          throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(1, UUID.randomUUID().toString());
+
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.INVALID_PART,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithMissingParts() throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(3, "random");

Review comment:
       The comment should be updated, this case is passing with an incorrect part number.
   
   

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
##########
@@ -351,8 +267,151 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       LOG.error("Unrecognized Result for S3MultipartUploadCommitRequest: {}",
           multipartUploadCompleteRequest);
     }
+  }
 
-    return omClientResponse;
+  @SuppressWarnings("checkstyle:ParameterNumber")
+  protected OmKeyInfo getOmKeyInfo(OzoneManager ozoneManager, long trxnLogIndex,
+      KeyArgs keyArgs, String volumeName, String bucketName, String keyName,
+      String multipartKey, OMMetadataManager omMetadataManager,
+      String ozoneKey, TreeMap<Integer, PartKeyInfo> partKeyInfoMap,
+      List<OmKeyLocationInfo> partLocationInfos, long dataSize)
+          throws IOException {
+    HddsProtos.ReplicationType type = partKeyInfoMap.lastEntry().getValue()
+        .getPartKeyInfo().getType();
+    HddsProtos.ReplicationFactor factor =
+        partKeyInfoMap.lastEntry().getValue().getPartKeyInfo().getFactor();
+
+    OmKeyInfo omKeyInfo = getOmKeyInfoFromKeyTable(ozoneKey, keyName,
+            omMetadataManager);
+    if (omKeyInfo == null) {
+      // This is a newly added key, it does not have any versions.
+      OmKeyLocationInfoGroup keyLocationInfoGroup = new
+          OmKeyLocationInfoGroup(0, partLocationInfos);
+
+      // Get the objectID of the key from OpenKeyTable
+      OmKeyInfo dbOpenKeyInfo = getOmKeyInfoFromOpenKeyTable(multipartKey,
+              keyName, omMetadataManager);
+
+      // A newly created key, this is the first version.
+      OmKeyInfo.Builder builder =
+          new OmKeyInfo.Builder().setVolumeName(volumeName)
+          .setBucketName(bucketName).setKeyName(dbOpenKeyInfo.getKeyName())
+          .setReplicationFactor(factor).setReplicationType(type)
+          .setCreationTime(keyArgs.getModificationTime())
+          .setModificationTime(keyArgs.getModificationTime())
+          .setDataSize(dataSize)
+          .setOmKeyLocationInfos(
+              Collections.singletonList(keyLocationInfoGroup))
+          .setAcls(OzoneAclUtil.fromProtobuf(keyArgs.getAclsList()));
+      // Check if db entry has ObjectID. This check is required because
+      // it is possible that between multipart key uploads and complete,
+      // we had an upgrade.
+      if (dbOpenKeyInfo.getObjectID() != 0) {
+        builder.setObjectID(dbOpenKeyInfo.getObjectID());
+      }
+      builder.setParentObjectID(dbOpenKeyInfo.getParentObjectID());
+      builder.setFileName(dbOpenKeyInfo.getFileName());

Review comment:
       Can we add isBucketFSOptimized() check when we setting the parentObjectID/FileName? These two field are only used when BucketFSO is enabled.

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadV1.java
##########
@@ -264,12 +274,232 @@ public void testUploadPartOverrideWithRatis() throws IOException {
     commitUploadPartInfo = ozoneOutputStream
             .getCommitUploadPartInfo();
 
-    assertNotNull(commitUploadPartInfo);
-    assertNotNull(commitUploadPartInfo.getPartName());
+    Assert.assertNotNull(commitUploadPartInfo);
+    Assert.assertNotNull(commitUploadPartInfo.getPartName());
 
     // PartName should be different from old part Name.
-    assertNotEquals("Part names should be different", partName,
+    Assert.assertNotEquals("Part names should be different", partName,
             commitUploadPartInfo.getPartName());
   }
 
+  @Test
+  public void testMultipartUploadWithPartsLessThanMinSize() throws Exception {

Review comment:
       Can we additionally add one another case for OMException.ResultCodes.INVALID_PART_ORDER? I didn't this case was covered in this unit test.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] rakeshadr commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577446016



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadV1.java
##########
@@ -264,12 +274,232 @@ public void testUploadPartOverrideWithRatis() throws IOException {
     commitUploadPartInfo = ozoneOutputStream
             .getCommitUploadPartInfo();
 
-    assertNotNull(commitUploadPartInfo);
-    assertNotNull(commitUploadPartInfo.getPartName());
+    Assert.assertNotNull(commitUploadPartInfo);
+    Assert.assertNotNull(commitUploadPartInfo.getPartName());
 
     // PartName should be different from old part Name.
-    assertNotEquals("Part names should be different", partName,
+    Assert.assertNotEquals("Part names should be different", partName,
             commitUploadPartInfo.getPartName());
   }
 
+  @Test
+  public void testMultipartUploadWithPartsLessThanMinSize() throws Exception {

Review comment:
       Sure. As `bucket.completeMultipartUpload(keyName, uploadID, partsMap); ` used hashmap to store parts, which will always be in sorted order. Inorder to make part list with incorrect order, I have added UT `testInvalidPartOrderError` in `TestS3MultipartUploadCompleteRequest` class.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] rakeshadr commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577576297



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java
##########
@@ -147,8 +148,18 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
-      String ozoneKey = omMetadataManager.getOzonePathKey(parentID, fileName);
-      partName = ozoneKey + clientID;
+      /**
+       * Format of PartName stored into MultipartInfoTable is,
+       * "fileName + ClientID".
+       *
+       * Contract is that all part names present in a multipart info will
+       * have same key prefix path.
+       *
+       * For example:
+       *        /vol1/buck1/a/b/c/part-1, /vol1/buck1/a/b/c/part-2,
+       *        /vol1/buck1/a/b/c/part-n
+       */
+      dbPartName = fileName + clientID;

Review comment:
       In the V0 (existing code), `PartName` is constructed using the `partName = ozoneKey + clientID;`.
   
   Now, in FSOBucket feature code, we store only the `fileName` in the `fileTable` and reconstructed `fullKeyName` while reading back from it. I thought of following the same format pattern in `multipartFileInfoTable` as well. Yes, like you said, we need to reconstruct it during #complete or #abort or #listParts etc.
   
   Following is the PartKeyInfo proto object. There we can see KeyInfo, which inturn has `parentID` and is having similar structure with other V1(FSOBucket feature) tables. I think, it would be helpful if we could follow same pattern everywhere in the new tables for better debugging unless if there is any functional challenges. Does that make sense to you?
   ```
   message PartKeyInfo {
       required string partName = 1;
       required uint32 partNumber = 2;
       required KeyInfo partKeyInfo = 3;
   }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] linyiqun commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577695818



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java
##########
@@ -147,8 +148,18 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
-      String ozoneKey = omMetadataManager.getOzonePathKey(parentID, fileName);
-      partName = ozoneKey + clientID;
+      /**
+       * Format of PartName stored into MultipartInfoTable is,
+       * "fileName + ClientID".
+       *
+       * Contract is that all part names present in a multipart info will
+       * have same key prefix path.
+       *
+       * For example:
+       *        /vol1/buck1/a/b/c/part-1, /vol1/buck1/a/b/c/part-2,
+       *        /vol1/buck1/a/b/c/part-n
+       */
+      dbPartName = fileName + clientID;

Review comment:
       Thanks @rakeshadr for very detailed explanation, make the sense to me.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] rakeshadr commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577440030



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadV1.java
##########
@@ -264,12 +274,232 @@ public void testUploadPartOverrideWithRatis() throws IOException {
     commitUploadPartInfo = ozoneOutputStream
             .getCommitUploadPartInfo();
 
-    assertNotNull(commitUploadPartInfo);
-    assertNotNull(commitUploadPartInfo.getPartName());
+    Assert.assertNotNull(commitUploadPartInfo);
+    Assert.assertNotNull(commitUploadPartInfo.getPartName());
 
     // PartName should be different from old part Name.
-    assertNotEquals("Part names should be different", partName,
+    Assert.assertNotEquals("Part names should be different", partName,
             commitUploadPartInfo.getPartName());
   }
 
+  @Test
+  public void testMultipartUploadWithPartsLessThanMinSize() throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    // Initiate multipart upload
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    // Upload Parts
+    Map<Integer, String> partsMap = new TreeMap<>();
+    // Uploading part 1 with less than min size
+    String partName = uploadPart(bucket, keyName, uploadID, 1,
+            "data".getBytes(UTF_8));
+    partsMap.put(1, partName);
+
+    partName = uploadPart(bucket, keyName, uploadID, 2,
+            "data".getBytes(UTF_8));
+    partsMap.put(2, partName);
+
+    // Complete multipart upload
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.ENTITY_TOO_SMALL,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithPartsMisMatchWithListSizeDifferent()
+          throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(1, UUID.randomUUID().toString());
+
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.INVALID_PART,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithPartsMisMatchWithIncorrectPartName()
+          throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(1, UUID.randomUUID().toString());

Review comment:
       Done!

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadV1.java
##########
@@ -264,12 +274,232 @@ public void testUploadPartOverrideWithRatis() throws IOException {
     commitUploadPartInfo = ozoneOutputStream
             .getCommitUploadPartInfo();
 
-    assertNotNull(commitUploadPartInfo);
-    assertNotNull(commitUploadPartInfo.getPartName());
+    Assert.assertNotNull(commitUploadPartInfo);
+    Assert.assertNotNull(commitUploadPartInfo.getPartName());
 
     // PartName should be different from old part Name.
-    assertNotEquals("Part names should be different", partName,
+    Assert.assertNotEquals("Part names should be different", partName,
             commitUploadPartInfo.getPartName());
   }
 
+  @Test
+  public void testMultipartUploadWithPartsLessThanMinSize() throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    // Initiate multipart upload
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    // Upload Parts
+    Map<Integer, String> partsMap = new TreeMap<>();
+    // Uploading part 1 with less than min size
+    String partName = uploadPart(bucket, keyName, uploadID, 1,
+            "data".getBytes(UTF_8));
+    partsMap.put(1, partName);
+
+    partName = uploadPart(bucket, keyName, uploadID, 2,
+            "data".getBytes(UTF_8));
+    partsMap.put(2, partName);
+
+    // Complete multipart upload
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.ENTITY_TOO_SMALL,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithPartsMisMatchWithListSizeDifferent()
+          throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(1, UUID.randomUUID().toString());
+
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.INVALID_PART,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithPartsMisMatchWithIncorrectPartName()
+          throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(1, UUID.randomUUID().toString());
+
+    OzoneTestUtils.expectOmException(OMException.ResultCodes.INVALID_PART,
+        () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
+  }
+
+  @Test
+  public void testMultipartUploadWithMissingParts() throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    String uploadID = initiateMultipartUpload(bucket, keyName, STAND_ALONE,
+            ONE);
+
+    uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
+
+    // We have not uploaded any parts, but passing some list it should throw
+    // error.
+    TreeMap<Integer, String> partsMap = new TreeMap<>();
+    partsMap.put(3, "random");

Review comment:
       Done!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] rakeshadr commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577677611



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java
##########
@@ -147,8 +148,18 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
-      String ozoneKey = omMetadataManager.getOzonePathKey(parentID, fileName);
-      partName = ozoneKey + clientID;
+      /**
+       * Format of PartName stored into MultipartInfoTable is,
+       * "fileName + ClientID".
+       *
+       * Contract is that all part names present in a multipart info will
+       * have same key prefix path.
+       *
+       * For example:
+       *        /vol1/buck1/a/b/c/part-1, /vol1/buck1/a/b/c/part-2,
+       *        /vol1/buck1/a/b/c/part-n
+       */
+      dbPartName = fileName + clientID;

Review comment:
       @linyiqun Yes, there will be multiple parts during the upload operations. Finally openFileTable, multiPartFileInfoTable entries will be deleted and then add fileTable entry corresponding to the user given KeyName. I have tried adding internals especially focussing on DB operations with an example for better understanding, can you please go through it and hope that helps to proceed further. Thanks a lot!
   
   KeyName = "a/b/c/file1". 
   Assume there are three parts, which will be having part number assigned like, 1, 2 and 3.
   
   **User Operation-1)** S3InitiateMultipartUpload : KeyName="a/b/c/file1"
   ```
           1) UploadID will be generated by OM : UUID.randomUUID().toString() + "-" + UniqueId.next()
   	2) Add "a/b/c" parent dirs into DirectoryTable
   	3) Add entry to openFileTable:
   			TableKey = multipartKey, which is nothing but <parentId/fileName/uploadId>
   			TableValue = omKeyInfo, which will store fileName and parentID.
   	4) Add entry to multipartFileInfoTable:		
   			TableKey = multipartKey, which is nothing but <parentId/fileName/uploadId>
   			TableValue = omKeyInfo, which will store uploadID and parentID. Since there is no real upload started, partKeyInfoList will be empty.
   	
   	Say, uploadID = 12345-0000. This is the unique identifier for uploading parts for the corresponding keyname.
   ```
   
   
   **User Operation-2)** S3CreateMultipartKey : KeyName="a/b/c/file1", PartNumber=1, uploadID="12345-0000"
   ```
   	1) ClientID will be generated by OM : UniqueId.next()
   	2) Since it has uploadID, it will check OmKeyInfo in openFileTable.
   	2) Add entry to OpenFileTable:
   			TableKey = openFileKey, which is nothing but <parentId/fileName/clientID>
   			TableValue = omKeyInfo, which will store fileName and parentID.
   	
   	This returns KeyOutputStream with unique ClientID. Say, ClientID=9777 and user streams data to it.
   	Finally during close, will invoke commitMultipartUploadPart.
   ```
   
   **User Operation-3)** S3CommitMultipartUploadPart: KeyName="a/b/c/file1", PartNumber=1, uploadID="12345-0000", ClientID=9777
   ```
   	1) Get <parentId/fileName/clientID> entry from openFileTable. Say, we got OmKeyInfo1 object from DB.
   	2) Create 'PartKeyInfo' using the OmKeyInfo1 object.
   			PartNumber = 1
   			PartName   = file1 + 9777; // fileName + ClientID;
   			keyInfo    = OmKeyInfo1    // here it stores only the fileName and parentID.
   	3) Get multipartKey <parentId/fileName/uploadId> entry from multipartFileInfoTable. Say, we got multipartKeyInfo1 object from DB.
   	4) Append/Add 'PartKeyInfo' to multipartKeyInfo1 object like,
   			multipartKeyInfo.addPartKeyInfo(partNumber, partKeyInfo.build());
   	5) Update the above multipartKeyInfo into the multipartFileInfoTable.
   	6) Delete <parentId/fileName/clientID> entry from openFileTable.
   ```
   	
   
   **User Operation-4)** 
   ```
          Repeat S3CreateMultipartKey and S3CommitMultipartUploadPart for PartNumber=2 and PartNumber=3.
           After these three parts upload operations, now the 'multipartKeyInfo' entry in multipartFileInfoTable will have three 'PartKeyInfo' items in the list.
   ```
   
   **User Operation-5)** S3CompleteMultipartUpload: KeyName="a/b/c/file1", partsMap={1,2,3}, uploadID="12345-0000".
   ```
   	1) Get multipartKey <parentId/fileName/uploadId> entry from multipartFileInfoTable. Say, we got multipartKeyInfo1 object from DB.
   	2) Compare all the user given part numbers and data size calculation etc using all the parts. I'm not exploring those info now as it is existing logic.
   	3) Prepare OmKeyInfo using <parentId/fileName> and add new entry into FileTable. This is now become the reference to the user given KeyName="a/b/c/file1".
   	4) Delete multipartKey <parentId/fileName/uploadId> entry from openFileTable.
   	5) Delete multipartKey <parentId/fileName/uploadId> entry from multipartFileInfoTable.
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] linyiqun commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577592666



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java
##########
@@ -147,8 +148,18 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
-      String ozoneKey = omMetadataManager.getOzonePathKey(parentID, fileName);
-      partName = ozoneKey + clientID;
+      /**
+       * Format of PartName stored into MultipartInfoTable is,
+       * "fileName + ClientID".
+       *
+       * Contract is that all part names present in a multipart info will
+       * have same key prefix path.
+       *
+       * For example:
+       *        /vol1/buck1/a/b/c/part-1, /vol1/buck1/a/b/c/part-2,
+       *        /vol1/buck1/a/b/c/part-n
+       */
+      dbPartName = fileName + clientID;

Review comment:
       @rakeshadr , I see current format  in multipartFileInfoTable, openFileTable, FileTable is like below.
   
      * |  fileTable         | parentId/fileName -> KeyInfo                    |
      * |----------------------------------------------------------------------|
      * |  openFileTable     | parentId/fileName/id -> KeyInfo                 |
      * |----------------------------------------------------------------------|
      * |  multipartFileInfoTable | parentId/fileName/uploadId ->...  
   
   
    So the format of part name is already follow above pattern now. So still need to do above part name change here?
   ```java
   String ozoneKey = omMetadataManager.getOzonePathKey(parentID, fileName);
   partName = ozoneKey + clientID;
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] rakeshadr merged pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
rakeshadr merged pull request #1923:
URL: https://github.com/apache/ozone/pull/1923


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] linyiqun commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577558074



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java
##########
@@ -147,8 +148,18 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
-      String ozoneKey = omMetadataManager.getOzonePathKey(parentID, fileName);
-      partName = ozoneKey + clientID;
+      /**
+       * Format of PartName stored into MultipartInfoTable is,
+       * "fileName + ClientID".
+       *
+       * Contract is that all part names present in a multipart info will
+       * have same key prefix path.
+       *
+       * For example:
+       *        /vol1/buck1/a/b/c/part-1, /vol1/buck1/a/b/c/part-2,
+       *        /vol1/buck1/a/b/c/part-n
+       */
+      dbPartName = fileName + clientID;

Review comment:
       Any specific reason for changing the format of part name?  With above format change, we have to get a full part name again in S3MultipartUploadCompleteRequestV1 request.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] linyiqun commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577592666



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java
##########
@@ -147,8 +148,18 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
-      String ozoneKey = omMetadataManager.getOzonePathKey(parentID, fileName);
-      partName = ozoneKey + clientID;
+      /**
+       * Format of PartName stored into MultipartInfoTable is,
+       * "fileName + ClientID".
+       *
+       * Contract is that all part names present in a multipart info will
+       * have same key prefix path.
+       *
+       * For example:
+       *        /vol1/buck1/a/b/c/part-1, /vol1/buck1/a/b/c/part-2,
+       *        /vol1/buck1/a/b/c/part-n
+       */
+      dbPartName = fileName + clientID;

Review comment:
       >Now, in FSOBucket feature code, we store only the fileName in the fileTable and reconstructed fullKeyName while reading back from it.
   
   @rakeshadr , I see current format  in multipartFileInfoTable, openFileTable, FileTable is like below. Above comment means that fileName is stored in KeyInfo? So we also only store part name(fileName + clientID;) in PartInfo?
   
   
      * |  fileTable         | parentId/fileName -> KeyInfo                    |
      * |----------------------------------------------------------------------|
      * |  openFileTable     | parentId/fileName/id -> KeyInfo                 |
      * |----------------------------------------------------------------------|
      * |  multipartFileInfoTable | parentId/fileName/uploadId ->...  
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] rakeshadr commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577446119



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
##########
@@ -351,8 +267,151 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       LOG.error("Unrecognized Result for S3MultipartUploadCommitRequest: {}",
           multipartUploadCompleteRequest);
     }
+  }
 
-    return omClientResponse;
+  @SuppressWarnings("checkstyle:ParameterNumber")
+  protected OmKeyInfo getOmKeyInfo(OzoneManager ozoneManager, long trxnLogIndex,
+      KeyArgs keyArgs, String volumeName, String bucketName, String keyName,
+      String multipartKey, OMMetadataManager omMetadataManager,
+      String ozoneKey, TreeMap<Integer, PartKeyInfo> partKeyInfoMap,
+      List<OmKeyLocationInfo> partLocationInfos, long dataSize)
+          throws IOException {
+    HddsProtos.ReplicationType type = partKeyInfoMap.lastEntry().getValue()
+        .getPartKeyInfo().getType();
+    HddsProtos.ReplicationFactor factor =
+        partKeyInfoMap.lastEntry().getValue().getPartKeyInfo().getFactor();
+
+    OmKeyInfo omKeyInfo = getOmKeyInfoFromKeyTable(ozoneKey, keyName,
+            omMetadataManager);
+    if (omKeyInfo == null) {
+      // This is a newly added key, it does not have any versions.
+      OmKeyLocationInfoGroup keyLocationInfoGroup = new
+          OmKeyLocationInfoGroup(0, partLocationInfos);
+
+      // Get the objectID of the key from OpenKeyTable
+      OmKeyInfo dbOpenKeyInfo = getOmKeyInfoFromOpenKeyTable(multipartKey,
+              keyName, omMetadataManager);
+
+      // A newly created key, this is the first version.
+      OmKeyInfo.Builder builder =
+          new OmKeyInfo.Builder().setVolumeName(volumeName)
+          .setBucketName(bucketName).setKeyName(dbOpenKeyInfo.getKeyName())
+          .setReplicationFactor(factor).setReplicationType(type)
+          .setCreationTime(keyArgs.getModificationTime())
+          .setModificationTime(keyArgs.getModificationTime())
+          .setDataSize(dataSize)
+          .setOmKeyLocationInfos(
+              Collections.singletonList(keyLocationInfoGroup))
+          .setAcls(OzoneAclUtil.fromProtobuf(keyArgs.getAclsList()));
+      // Check if db entry has ObjectID. This check is required because
+      // it is possible that between multipart key uploads and complete,
+      // we had an upgrade.
+      if (dbOpenKeyInfo.getObjectID() != 0) {
+        builder.setObjectID(dbOpenKeyInfo.getObjectID());
+      }
+      builder.setParentObjectID(dbOpenKeyInfo.getParentObjectID());
+      builder.setFileName(dbOpenKeyInfo.getFileName());

Review comment:
       Done!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] rakeshadr commented on a change in pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#discussion_r577576297



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java
##########
@@ -147,8 +148,18 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
-      String ozoneKey = omMetadataManager.getOzonePathKey(parentID, fileName);
-      partName = ozoneKey + clientID;
+      /**
+       * Format of PartName stored into MultipartInfoTable is,
+       * "fileName + ClientID".
+       *
+       * Contract is that all part names present in a multipart info will
+       * have same key prefix path.
+       *
+       * For example:
+       *        /vol1/buck1/a/b/c/part-1, /vol1/buck1/a/b/c/part-2,
+       *        /vol1/buck1/a/b/c/part-n
+       */
+      dbPartName = fileName + clientID;

Review comment:
       In the V0 (existing code), `PartName` is constructed using the `partName = ozoneKey + clientID;`.
   
   Now, in FSOBucket feature code, we store only the `fileName` in the `fileTable` and reconstructed `fullKeyName` while reading back from it. I thought of following the same format pattern the `multipartFileInfoTable` as well. Yes, like you said, we need to reconstruct it during #complete or #abort or #listParts etc.
   
   Following is the PartKeyInfo proto object. There we can see KeyInfo, which inturn has `parentID` and is having similar structure with other V1(FSOBucket feature) tables. I think, it would be helpful if we could follow same pattern everywhere in the new tables for better debugging unless if there is any functional challenges. Does that make sense to you?
   ```
   message PartKeyInfo {
       required string partName = 1;
       required uint32 partNumber = 2;
       required KeyInfo partKeyInfo = 3;
   }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] rakeshadr commented on pull request #1923: HDDS-4813. [FSO]S3Multipart: Implement UploadCompleteRequest

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on pull request #1923:
URL: https://github.com/apache/ozone/pull/1923#issuecomment-780628285


   Thanks a lot @linyiqun for the reviews and useful comments. I'm merging it to the branch.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org