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/05/10 10:18:07 UTC

[GitHub] [ozone] rakeshadr opened a new pull request #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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


   ## What changes were proposed in this pull request?
   
   This task is to simplify the prefix layout implementation for the S3MPU keys. As there is no ObjectStore API to perform rename or delete operation on MPU keys, the DB table key format can follow the existing structure for the MPU table entries.
   
   This will help to reuse the existing ozoneBucket#listParts() and ozoneBucket#listMultipartUploads() and required very minimal code changes.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5201
   
   ## How was this patch tested?
   
   Added and modify unit & integration test 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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartUploadCommitPartRequestWithFSO.java
##########
@@ -66,13 +66,20 @@ protected void addKeyToOpenKeyTable(String volumeName, String bucketName,
             fileName, omKeyInfo, clientID, txnLogId, omMetadataManager);
   }
 
-  protected String getMultipartKey(String volumeName, String bucketName,
+  protected String getMultipartOpenKey(String volumeName, String bucketName,

Review comment:
       Same comment for adding `@Override`. 
   @rakeshadr , can you check similar other places and fix them all in **WithFSO.class? As currently most of logic can be reused, it will be better to add `override`.




-- 
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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequestWithFSO.java
##########
@@ -54,19 +54,20 @@ protected OMClientResponse getOmClientResponse(IOException exception,
 
   protected OMClientResponse getOmClientResponse(OzoneManager ozoneManager,
       OmMultipartKeyInfo multipartKeyInfo, String multipartKey,
-      OMResponse.Builder omResponse, OmBucketInfo omBucketInfo) {
+      String multipartOpenKey, OMResponse.Builder omResponse,
+      OmBucketInfo omBucketInfo) {
 
     OMClientResponse omClientResp = new S3MultipartUploadAbortResponseWithFSO(
         omResponse.setAbortMultiPartUploadResponse(
-            MultipartUploadAbortResponse.newBuilder()).build(),
-        multipartKey, multipartKeyInfo, ozoneManager.isRatisEnabled(),
+            MultipartUploadAbortResponse.newBuilder()).build(), multipartKey,
+        multipartOpenKey, multipartKeyInfo, ozoneManager.isRatisEnabled(),
         omBucketInfo.copyObject());
     return omClientResp;
   }
 
-  protected String getMultipartKey(String multipartUploadID, String volumeName,
-      String bucketName, String keyName, OMMetadataManager omMetadataManager)
-      throws IOException {
+  protected String getMultipartOpenKey(String multipartUploadID,

Review comment:
       Can we add `@Override` for this method?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestWithFSO.java
##########
@@ -62,197 +46,41 @@ public S3MultipartUploadCommitPartRequestWithFSO(OMRequest omRequest) {
   }
 
   @Override
-  @SuppressWarnings("methodlength")
-  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
-      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
-    MultipartCommitUploadPartRequest multipartCommitUploadPartRequest =
-        getOmRequest().getCommitMultiPartUploadRequest();
-
-    KeyArgs keyArgs = multipartCommitUploadPartRequest.getKeyArgs();
-    Map<String, String> auditMap = buildKeyArgsAuditMap(keyArgs);
-
-    String volumeName = keyArgs.getVolumeName();
-    String bucketName = keyArgs.getBucketName();
-    String keyName = keyArgs.getKeyName();
-
-    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
-    ozoneManager.getMetrics().incNumCommitMultipartUploadParts();
-
-    boolean acquiredLock = false;
-
-    IOException exception = null;
-    String dbPartName;
-    String fullKeyPartName = null;
-    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
-        getOmRequest());
-    OMClientResponse omClientResponse = null;
-    OzoneManagerProtocolProtos.PartKeyInfo oldPartKeyInfo = null;
-    String openFileKey = null;
-    OmKeyInfo omKeyInfo = null;
-    String multipartKey = null;
-    OmMultipartKeyInfo multipartKeyInfo = null;
-    Result result;
-    OmBucketInfo omBucketInfo;
-    OmBucketInfo copyBucketInfo = null;
-    try {
-      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
-      volumeName = keyArgs.getVolumeName();
-      bucketName = keyArgs.getBucketName();
-
-      // TODO to support S3 ACL later.
-      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
-          volumeName, bucketName);
-
-      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
-
-      String fileName = OzoneFSUtils.getFileName(keyName);
-      Iterator<Path> pathComponents = Paths.get(keyName).iterator();
-      String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
-      omBucketInfo = omMetadataManager.getBucketTable().get(bucketKey);
-      long bucketId = omBucketInfo.getObjectID();
-      long parentID = OMFileRequest.getParentID(bucketId, pathComponents,
-              keyName, omMetadataManager);
-
-      String uploadID = keyArgs.getMultipartUploadID();
-      multipartKey = omMetadataManager.getMultipartKey(parentID,
-          fileName, uploadID);
-
-      multipartKeyInfo = omMetadataManager.getMultipartInfoTable()
-          .get(multipartKey);
-
-      long clientID = multipartCommitUploadPartRequest.getClientID();
-
-      openFileKey = omMetadataManager.getOpenFileName(parentID, fileName,
-          clientID);
-
-      omKeyInfo = OMFileRequest.getOmKeyInfoFromFileTable(true,
-              omMetadataManager, openFileKey, keyName);
-
-      if (omKeyInfo == null) {
-        throw new OMException("Failed to commit Multipart Upload key, as " +
-            openFileKey + " entry is not found in the openFileTable",
-            KEY_NOT_FOUND);
-      }
-
-      // set the data size and location info list
-      omKeyInfo.setDataSize(keyArgs.getDataSize());
-      omKeyInfo.updateLocationInfoList(keyArgs.getKeyLocationsList().stream()
-          .map(OmKeyLocationInfo::getFromProtobuf)
-          .collect(Collectors.toList()), true);
-      // Set Modification time
-      omKeyInfo.setModificationTime(keyArgs.getModificationTime());
-      // Set the UpdateID to current transactionLogIndex
-      omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
-
-      /**
-       * Format of PartName stored into MultipartInfoTable is,
-       * "<parentID>/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
-       */
-      String ozoneFileKey = omMetadataManager.getOzonePathKey(parentID,
-          fileName);
-      dbPartName = ozoneFileKey + clientID;
-
-      if (multipartKeyInfo == null) {
-        // This can occur when user started uploading part by the time commit
-        // of that part happens, in between the user might have requested
-        // abort multipart upload. If we just throw exception, then the data
-        // will not be garbage collected, so move this part to delete table
-        // and throw error
-        // Move this part to delete table.
-        throw new OMException("No such Multipart upload is with specified " +
-            "uploadId " + uploadID,
-            OMException.ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
-      }
-
-      int partNumber = keyArgs.getMultipartNumber();
-      oldPartKeyInfo = multipartKeyInfo.getPartKeyInfo(partNumber);
-
-      // Build this multipart upload part info.
-      OzoneManagerProtocolProtos.PartKeyInfo.Builder partKeyInfo =
-          OzoneManagerProtocolProtos.PartKeyInfo.newBuilder();
-      partKeyInfo.setPartName(dbPartName);
-      partKeyInfo.setPartNumber(partNumber);
-      partKeyInfo.setPartKeyInfo(omKeyInfo.getProtobuf(fileName,
-          getOmRequest().getVersion()));
-
-      // Add this part information in to multipartKeyInfo.
-      multipartKeyInfo.addPartKeyInfo(partNumber, partKeyInfo.build());
-
-      // Set the UpdateID to current transactionLogIndex
-      multipartKeyInfo.setUpdateID(trxnLogIndex,
-          ozoneManager.isRatisEnabled());
-
-      // OldPartKeyInfo will be deleted. Its updateID will be set in
-      // S3MultipartUplodaCommitPartResponse before being added to
-      // DeletedKeyTable.
-
-      // Delete from open key table and add it to multipart info table.
-      // No need to add cache entries to delete table, as no
-      // read/write requests that info for validation.
-      omMetadataManager.getMultipartInfoTable().addCacheEntry(
-          new CacheKey<>(multipartKey),
-          new CacheValue<>(Optional.of(multipartKeyInfo),
-              trxnLogIndex));
-
-      omMetadataManager.getOpenKeyTable().addCacheEntry(
-          new CacheKey<>(openFileKey),
-          new CacheValue<>(Optional.absent(), trxnLogIndex));
-
-      long scmBlockSize = ozoneManager.getScmBlockSize();
-      int factor = omKeyInfo.getFactor().getNumber();
-      omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
-      // Block was pre-requested and UsedBytes updated when createKey and
-      // AllocatedBlock. The space occupied by the Key shall be based on
-      // the actual Key size, and the total Block size applied before should
-      // be subtracted.
-      long correctedSpace = omKeyInfo.getDataSize() * factor -
-          keyArgs.getKeyLocationsList().size() * scmBlockSize * factor;
-      omBucketInfo.incrUsedBytes(correctedSpace);
-
-      // Prepare response. Sets user given full key part name in 'partName'
-      // attribute in response object.
-      String fullOzoneKeyName = omMetadataManager.getOzoneKey(
-              volumeName, bucketName, keyName);
-      fullKeyPartName = fullOzoneKeyName + clientID;
-      omResponse.setCommitMultiPartUploadResponse(
-          MultipartCommitUploadPartResponse.newBuilder()
-              .setPartName(fullKeyPartName));
-
-      omClientResponse = new S3MultipartUploadCommitPartResponseWithFSO(
-          omResponse.build(), multipartKey, openFileKey,
-          multipartKeyInfo, oldPartKeyInfo, omKeyInfo,
-          ozoneManager.isRatisEnabled(),
-          omBucketInfo.copyObject());
+  protected String getOpenKey(String volumeName, String bucketName,
+      String keyName, OMMetadataManager omMetadataManager, long clientID)
+      throws IOException {
+
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+        omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    long parentID = OMFileRequest
+        .getParentID(bucketId, pathComponents, keyName, omMetadataManager);
+
+    return omMetadataManager.getOpenFileName(parentID, fileName, clientID);
+  }
 
-      result = Result.SUCCESS;
-    } catch (IOException ex) {
-      result = Result.FAILURE;
-      exception = ex;
-      omClientResponse = new S3MultipartUploadCommitPartResponseWithFSO(
-          createErrorOMResponse(omResponse, exception), multipartKey,
-          openFileKey, multipartKeyInfo, oldPartKeyInfo, omKeyInfo,
-          ozoneManager.isRatisEnabled(), copyBucketInfo);
-    } finally {
-      addResponseToDoubleBuffer(trxnLogIndex, omClientResponse,
-          omDoubleBufferHelper);
-      if (acquiredLock) {
-        omMetadataManager.getLock().releaseWriteLock(BUCKET_LOCK,
-            volumeName, bucketName);
-      }
-    }
+  @Override
+  protected OmKeyInfo getOmKeyInfo(OMMetadataManager omMetadataManager,
+      String openKey, String keyName) throws IOException {
 
-    logResult(ozoneManager, multipartCommitUploadPartRequest, keyArgs,
-            auditMap, volumeName, bucketName, keyName, exception,
-            fullKeyPartName, result);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+        omMetadataManager, openKey, keyName);
+  }
 
-    return omClientResponse;
+  @SuppressWarnings("checkstyle:ParameterNumber")
+  protected S3MultipartUploadCommitPartResponse getOmClientResponse(

Review comment:
       Can also add `@Override` here?

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartUploadCommitPartRequestWithFSO.java
##########
@@ -66,13 +66,20 @@ protected void addKeyToOpenKeyTable(String volumeName, String bucketName,
             fileName, omKeyInfo, clientID, txnLogId, omMetadataManager);
   }
 
-  protected String getMultipartKey(String volumeName, String bucketName,
+  protected String getMultipartOpenKey(String volumeName, String bucketName,

Review comment:
       Same comment for adding `@Override`. 
   @rakeshadr , can you check similar other places and fix them all in **WithFSO.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 pull request #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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


   > @rakeshadr , thanks for addressing the review comment. Only catch one nit after more deep reviewed.
   > BTW, one latest CI check is failed. Please have a look.
   
   Thanks again @linyiqun for the reviews. The tests failures are unrelated. Multi-delete and s3, acceptance test are failing, these are running with simple layout version and are random failures. I've pushed one more commit and will monitor next CI run.


-- 
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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequestWithFSO.java
##########
@@ -54,19 +54,20 @@ protected OMClientResponse getOmClientResponse(IOException exception,
 
   protected OMClientResponse getOmClientResponse(OzoneManager ozoneManager,
       OmMultipartKeyInfo multipartKeyInfo, String multipartKey,
-      OMResponse.Builder omResponse, OmBucketInfo omBucketInfo) {
+      String multipartOpenKey, OMResponse.Builder omResponse,
+      OmBucketInfo omBucketInfo) {
 
     OMClientResponse omClientResp = new S3MultipartUploadAbortResponseWithFSO(
         omResponse.setAbortMultiPartUploadResponse(
-            MultipartUploadAbortResponse.newBuilder()).build(),
-        multipartKey, multipartKeyInfo, ozoneManager.isRatisEnabled(),
+            MultipartUploadAbortResponse.newBuilder()).build(), multipartKey,
+        multipartOpenKey, multipartKeyInfo, ozoneManager.isRatisEnabled(),
         omBucketInfo.copyObject());
     return omClientResp;
   }
 
-  protected String getMultipartKey(String multipartUploadID, String volumeName,
-      String bucketName, String keyName, OMMetadataManager omMetadataManager)
-      throws IOException {
+  protected String getMultipartOpenKey(String multipartUploadID,

Review comment:
       Done!

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestWithFSO.java
##########
@@ -62,197 +46,41 @@ public S3MultipartUploadCommitPartRequestWithFSO(OMRequest omRequest) {
   }
 
   @Override
-  @SuppressWarnings("methodlength")
-  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
-      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
-    MultipartCommitUploadPartRequest multipartCommitUploadPartRequest =
-        getOmRequest().getCommitMultiPartUploadRequest();
-
-    KeyArgs keyArgs = multipartCommitUploadPartRequest.getKeyArgs();
-    Map<String, String> auditMap = buildKeyArgsAuditMap(keyArgs);
-
-    String volumeName = keyArgs.getVolumeName();
-    String bucketName = keyArgs.getBucketName();
-    String keyName = keyArgs.getKeyName();
-
-    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
-    ozoneManager.getMetrics().incNumCommitMultipartUploadParts();
-
-    boolean acquiredLock = false;
-
-    IOException exception = null;
-    String dbPartName;
-    String fullKeyPartName = null;
-    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
-        getOmRequest());
-    OMClientResponse omClientResponse = null;
-    OzoneManagerProtocolProtos.PartKeyInfo oldPartKeyInfo = null;
-    String openFileKey = null;
-    OmKeyInfo omKeyInfo = null;
-    String multipartKey = null;
-    OmMultipartKeyInfo multipartKeyInfo = null;
-    Result result;
-    OmBucketInfo omBucketInfo;
-    OmBucketInfo copyBucketInfo = null;
-    try {
-      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
-      volumeName = keyArgs.getVolumeName();
-      bucketName = keyArgs.getBucketName();
-
-      // TODO to support S3 ACL later.
-      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
-          volumeName, bucketName);
-
-      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
-
-      String fileName = OzoneFSUtils.getFileName(keyName);
-      Iterator<Path> pathComponents = Paths.get(keyName).iterator();
-      String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
-      omBucketInfo = omMetadataManager.getBucketTable().get(bucketKey);
-      long bucketId = omBucketInfo.getObjectID();
-      long parentID = OMFileRequest.getParentID(bucketId, pathComponents,
-              keyName, omMetadataManager);
-
-      String uploadID = keyArgs.getMultipartUploadID();
-      multipartKey = omMetadataManager.getMultipartKey(parentID,
-          fileName, uploadID);
-
-      multipartKeyInfo = omMetadataManager.getMultipartInfoTable()
-          .get(multipartKey);
-
-      long clientID = multipartCommitUploadPartRequest.getClientID();
-
-      openFileKey = omMetadataManager.getOpenFileName(parentID, fileName,
-          clientID);
-
-      omKeyInfo = OMFileRequest.getOmKeyInfoFromFileTable(true,
-              omMetadataManager, openFileKey, keyName);
-
-      if (omKeyInfo == null) {
-        throw new OMException("Failed to commit Multipart Upload key, as " +
-            openFileKey + " entry is not found in the openFileTable",
-            KEY_NOT_FOUND);
-      }
-
-      // set the data size and location info list
-      omKeyInfo.setDataSize(keyArgs.getDataSize());
-      omKeyInfo.updateLocationInfoList(keyArgs.getKeyLocationsList().stream()
-          .map(OmKeyLocationInfo::getFromProtobuf)
-          .collect(Collectors.toList()), true);
-      // Set Modification time
-      omKeyInfo.setModificationTime(keyArgs.getModificationTime());
-      // Set the UpdateID to current transactionLogIndex
-      omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
-
-      /**
-       * Format of PartName stored into MultipartInfoTable is,
-       * "<parentID>/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
-       */
-      String ozoneFileKey = omMetadataManager.getOzonePathKey(parentID,
-          fileName);
-      dbPartName = ozoneFileKey + clientID;
-
-      if (multipartKeyInfo == null) {
-        // This can occur when user started uploading part by the time commit
-        // of that part happens, in between the user might have requested
-        // abort multipart upload. If we just throw exception, then the data
-        // will not be garbage collected, so move this part to delete table
-        // and throw error
-        // Move this part to delete table.
-        throw new OMException("No such Multipart upload is with specified " +
-            "uploadId " + uploadID,
-            OMException.ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
-      }
-
-      int partNumber = keyArgs.getMultipartNumber();
-      oldPartKeyInfo = multipartKeyInfo.getPartKeyInfo(partNumber);
-
-      // Build this multipart upload part info.
-      OzoneManagerProtocolProtos.PartKeyInfo.Builder partKeyInfo =
-          OzoneManagerProtocolProtos.PartKeyInfo.newBuilder();
-      partKeyInfo.setPartName(dbPartName);
-      partKeyInfo.setPartNumber(partNumber);
-      partKeyInfo.setPartKeyInfo(omKeyInfo.getProtobuf(fileName,
-          getOmRequest().getVersion()));
-
-      // Add this part information in to multipartKeyInfo.
-      multipartKeyInfo.addPartKeyInfo(partNumber, partKeyInfo.build());
-
-      // Set the UpdateID to current transactionLogIndex
-      multipartKeyInfo.setUpdateID(trxnLogIndex,
-          ozoneManager.isRatisEnabled());
-
-      // OldPartKeyInfo will be deleted. Its updateID will be set in
-      // S3MultipartUplodaCommitPartResponse before being added to
-      // DeletedKeyTable.
-
-      // Delete from open key table and add it to multipart info table.
-      // No need to add cache entries to delete table, as no
-      // read/write requests that info for validation.
-      omMetadataManager.getMultipartInfoTable().addCacheEntry(
-          new CacheKey<>(multipartKey),
-          new CacheValue<>(Optional.of(multipartKeyInfo),
-              trxnLogIndex));
-
-      omMetadataManager.getOpenKeyTable().addCacheEntry(
-          new CacheKey<>(openFileKey),
-          new CacheValue<>(Optional.absent(), trxnLogIndex));
-
-      long scmBlockSize = ozoneManager.getScmBlockSize();
-      int factor = omKeyInfo.getFactor().getNumber();
-      omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
-      // Block was pre-requested and UsedBytes updated when createKey and
-      // AllocatedBlock. The space occupied by the Key shall be based on
-      // the actual Key size, and the total Block size applied before should
-      // be subtracted.
-      long correctedSpace = omKeyInfo.getDataSize() * factor -
-          keyArgs.getKeyLocationsList().size() * scmBlockSize * factor;
-      omBucketInfo.incrUsedBytes(correctedSpace);
-
-      // Prepare response. Sets user given full key part name in 'partName'
-      // attribute in response object.
-      String fullOzoneKeyName = omMetadataManager.getOzoneKey(
-              volumeName, bucketName, keyName);
-      fullKeyPartName = fullOzoneKeyName + clientID;
-      omResponse.setCommitMultiPartUploadResponse(
-          MultipartCommitUploadPartResponse.newBuilder()
-              .setPartName(fullKeyPartName));
-
-      omClientResponse = new S3MultipartUploadCommitPartResponseWithFSO(
-          omResponse.build(), multipartKey, openFileKey,
-          multipartKeyInfo, oldPartKeyInfo, omKeyInfo,
-          ozoneManager.isRatisEnabled(),
-          omBucketInfo.copyObject());
+  protected String getOpenKey(String volumeName, String bucketName,
+      String keyName, OMMetadataManager omMetadataManager, long clientID)
+      throws IOException {
+
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+        omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    long parentID = OMFileRequest
+        .getParentID(bucketId, pathComponents, keyName, omMetadataManager);
+
+    return omMetadataManager.getOpenFileName(parentID, fileName, clientID);
+  }
 
-      result = Result.SUCCESS;
-    } catch (IOException ex) {
-      result = Result.FAILURE;
-      exception = ex;
-      omClientResponse = new S3MultipartUploadCommitPartResponseWithFSO(
-          createErrorOMResponse(omResponse, exception), multipartKey,
-          openFileKey, multipartKeyInfo, oldPartKeyInfo, omKeyInfo,
-          ozoneManager.isRatisEnabled(), copyBucketInfo);
-    } finally {
-      addResponseToDoubleBuffer(trxnLogIndex, omClientResponse,
-          omDoubleBufferHelper);
-      if (acquiredLock) {
-        omMetadataManager.getLock().releaseWriteLock(BUCKET_LOCK,
-            volumeName, bucketName);
-      }
-    }
+  @Override
+  protected OmKeyInfo getOmKeyInfo(OMMetadataManager omMetadataManager,
+      String openKey, String keyName) throws IOException {
 
-    logResult(ozoneManager, multipartCommitUploadPartRequest, keyArgs,
-            auditMap, volumeName, bucketName, keyName, exception,
-            fullKeyPartName, result);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+        omMetadataManager, openKey, keyName);
+  }
 
-    return omClientResponse;
+  @SuppressWarnings("checkstyle:ParameterNumber")
+  protected S3MultipartUploadCommitPartResponse getOmClientResponse(

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] mukul1987 commented on pull request #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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


   i do not think, if we have a bucket rename API available right now.
   
   However with HDDS-2939, even if the bucket it renamed, its object id will still remain the same, so we will still be to use the code


-- 
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 pull request #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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


   >However with HDDS-2939, even if the bucket it renamed, its object id will still remain the same, so we will still be to use the code
   
   Yes, the object id stored in file/dir table can make sense here. But multipartInfoTable didn't stored the object id, it stores the full path as original logic. Maybe this is not a big problem now, I just bring up this corner case that I can Imagine.


-- 
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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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


   > Yes, the object id stored in file/dir table can make sense here. But multipartInfoTable didn't stored the object id, it stores the full path as original logic. Maybe this is not a big problem now, I just bring up this corner case that I can Imagine.
   
   @linyiqun Thanks for adding more details and its really a good thought. IMHO, since Ozone don't support bucket#rename API this case won't hit now. Also, I searched bucket#rename API in S3. I could see that they are not providing bucket#rename API in S3 and recommended users to do copy keys to a new bucket.
   
   


-- 
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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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


   **Note:** Since I have touched all the S3 flows - initiate -> commit -> complete and abort the patch has modified all these source and test classes.


-- 
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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartUploadCommitPartRequestWithFSO.java
##########
@@ -66,13 +66,20 @@ protected void addKeyToOpenKeyTable(String volumeName, String bucketName,
             fileName, omKeyInfo, clientID, txnLogId, omMetadataManager);
   }
 
-  protected String getMultipartKey(String volumeName, String bucketName,
+  protected String getMultipartOpenKey(String volumeName, String bucketName,

Review comment:
       Thanks @linyiqun for the comment. I have visited all the FSO files and I hope all the cases has been taken care now.




-- 
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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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


   


-- 
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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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


   > Haven't taken a detailed review, some initial comments below.
   > In addition, I have a question for this, @rakeshadr As you mentioned, MPU api cannot support renamed. But if there will be a corner case that S3 bucket can be renamed by admin user since currently we use specific bucket as the S3 bucket.
   > After S3 bucket be renamed, will current MPU logic be broken?
   
   Thanks @linyiqun for the review comments. 
   I'm adding @mukul1987 's comment link [here](https://github.com/apache/ozone/pull/2228/#issuecomment-838811615) to chain the discussion thread..


-- 
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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java
##########
@@ -846,8 +930,7 @@ private String verifyUploadedPart(String volumeName, String bucketName,
         omKeyInfo.getKeyName());
     Assert.assertEquals(uploadID, omMultipartKeyInfo.getUploadID());
 
-    long parentID = getParentID(volumeName, bucketName, keyName,
-        metadataMgr);
+    getParentID(volumeName, bucketName, keyName, metadataMgr);

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] linyiqun commented on a change in pull request #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java
##########
@@ -846,8 +930,7 @@ private String verifyUploadedPart(String volumeName, String bucketName,
         omKeyInfo.getKeyName());
     Assert.assertEquals(uploadID, omMultipartKeyInfo.getUploadID());
 
-    long parentID = getParentID(volumeName, bucketName, keyName,
-        metadataMgr);
+    getParentID(volumeName, bucketName, keyName, metadataMgr);

Review comment:
       Nit: This line can be removed.




-- 
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 #2228: HDDS-5201. [FSO] S3MultiPart: Use existing ozone key format for MPU Info in DB

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


   Thanks @linyiqun for the reviews.
   Thanks @mukul1987 , @bharatviswa504 , @sadanand48 for the offline discussions.
   
   I've merged the changes to the feature branch and with this hopefully s3 acceptance test would be able to run happily.


-- 
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