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/03/05 15:07:38 UTC

[GitHub] [ozone] rakeshadr opened a new pull request #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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


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


----------------------------------------------------------------
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 #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/multipart/S3MultipartUploadAbortResponseV1.java
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.response.s3.multipart;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.MULTIPARTFILEINFO_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
+
+/**
+ * Response for Multipart Abort Request layout version V1.
+ */
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, DELETED_TABLE,
+    MULTIPARTFILEINFO_TABLE})
+public class S3MultipartUploadAbortResponseV1
+    extends S3MultipartUploadAbortResponse {
+
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse,
+      String multipartKey, @Nonnull OmMultipartKeyInfo omMultipartKeyInfo,
+      boolean isRatisEnabled, @Nonnull OmBucketInfo omBucketInfo) {
+
+    super(omResponse, multipartKey, omMultipartKeyInfo, isRatisEnabled,
+        omBucketInfo);
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // Delete from openKey table and multipart info table.
+    omMetadataManager.getOpenKeyTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+    omMetadataManager.getMultipartInfoTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+
+    // Move all the parts to delete table
+    TreeMap<Integer, PartKeyInfo > partKeyInfoMap =
+        getOmMultipartKeyInfo().getPartKeyInfoMap();
+    for (Map.Entry<Integer, PartKeyInfo > partKeyInfoEntry :
+        partKeyInfoMap.entrySet()) {
+      PartKeyInfo partKeyInfo = partKeyInfoEntry.getValue();
+      OmKeyInfo currentKeyPartInfo =
+          OmKeyInfo.getFromProtobuf(partKeyInfo.getPartKeyInfo());
+
+      RepeatedOmKeyInfo repeatedOmKeyInfo =
+          omMetadataManager.getDeletedTable().get(partKeyInfo.getPartName());

Review comment:
       Thanks again for the detailed explanation. Good catch! @linyiqun 
   
   I have updated the partName logic and the partName format is now `<parentID>/fileName + ClientID`.
   
   Since the `partKeyInfo.getPartName()` stored in DB is in the above format all the DB operations will happen using this dbPartName. So, no specific conversion required in response class. 
   
   To the user after commit, it will return the` fullKeyPartName` . [Reference code:](https://github.com/apache/ozone/blob/HDDS-2939/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java#L225).




----------------------------------------------------------------
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 #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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


   > LGTM, +1.
   > As we use the new format key parentID/fileName + ClientID in delete table, suppose we might need to do some adjustment in current key delete service to support this type of key in follow-up task.
   
   Sure, we have to work on the deletion keys.
   
   Thanks once again @linyiqun for the continuous support by providing useful review comments. I will merge it.


----------------------------------------------------------------
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 edited a comment on pull request #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

Posted by GitBox <gi...@apache.org>.
rakeshadr edited a comment on pull request #1997:
URL: https://github.com/apache/ozone/pull/1997#issuecomment-793760320


   > LGTM, +1.
   > As we use the new format key parentID/fileName + ClientID in delete table, suppose we might need to do some adjustment in current key delete service to support this type of key in follow-up task.
   
   Sure, we have to work on the deletion keys. I will start analysing these cases on HDDS-4495 task.
   
   Thanks once again @linyiqun for the continuous support by providing useful review comments. I will merge it.


----------------------------------------------------------------
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 #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/multipart/S3MultipartUploadAbortResponseV1.java
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.response.s3.multipart;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.MULTIPARTFILEINFO_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
+
+/**
+ * Response for Multipart Abort Request layout version V1.
+ */
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, DELETED_TABLE,
+    MULTIPARTFILEINFO_TABLE})
+public class S3MultipartUploadAbortResponseV1
+    extends S3MultipartUploadAbortResponse {
+
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse,
+      String multipartKey, @Nonnull OmMultipartKeyInfo omMultipartKeyInfo,
+      boolean isRatisEnabled, @Nonnull OmBucketInfo omBucketInfo) {
+
+    super(omResponse, multipartKey, omMultipartKeyInfo, isRatisEnabled,
+        omBucketInfo);
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // Delete from openKey table and multipart info table.
+    omMetadataManager.getOpenKeyTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+    omMetadataManager.getMultipartInfoTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+
+    // Move all the parts to delete table
+    TreeMap<Integer, PartKeyInfo > partKeyInfoMap =
+        getOmMultipartKeyInfo().getPartKeyInfoMap();
+    for (Map.Entry<Integer, PartKeyInfo > partKeyInfoEntry :
+        partKeyInfoMap.entrySet()) {
+      PartKeyInfo partKeyInfo = partKeyInfoEntry.getValue();
+      OmKeyInfo currentKeyPartInfo =
+          OmKeyInfo.getFromProtobuf(partKeyInfo.getPartKeyInfo());
+
+      RepeatedOmKeyInfo repeatedOmKeyInfo =
+          omMetadataManager.getDeletedTable().get(partKeyInfo.getPartName());

Review comment:
       Thanks for addressing the comment, @rakeshadr .
   The reference UT verified that the part key entries are correctly put into the delete table.
   But the key name here might be a full part key name I think, as delete table format is like below:
   
   ```java
   |----------------------------------------------------------------------|
   | deletedTable       | /volumeName/bucketName/keyName->RepeatedKeyInfo |
   |----------------------------------------------------------------------|
   ```
   
   Now part name stored in omMultipartKeyInfo is filename+clientID, use this part name as a key name in delete table doesn't match above format.
   
   So below line needed to be update:
   ```java
         omMetadataManager.getDeletedTable().putWithBatch(batchOperation,
             partKeyInfo.getPartName(), repeatedOmKeyInfo);
   ```
   
   Correct me If I am wrong.




----------------------------------------------------------------
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 #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/multipart/S3MultipartUploadAbortResponseV1.java
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.response.s3.multipart;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.MULTIPARTFILEINFO_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
+
+/**
+ * Response for Multipart Abort Request layout version V1.
+ */
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, DELETED_TABLE,
+    MULTIPARTFILEINFO_TABLE})
+public class S3MultipartUploadAbortResponseV1
+    extends S3MultipartUploadAbortResponse {
+
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse,
+      String multipartKey, @Nonnull OmMultipartKeyInfo omMultipartKeyInfo,
+      boolean isRatisEnabled, @Nonnull OmBucketInfo omBucketInfo) {
+
+    super(omResponse, multipartKey, omMultipartKeyInfo, isRatisEnabled,
+        omBucketInfo);
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // Delete from openKey table and multipart info table.
+    omMetadataManager.getOpenKeyTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+    omMetadataManager.getMultipartInfoTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+
+    // Move all the parts to delete table
+    TreeMap<Integer, PartKeyInfo > partKeyInfoMap =
+        getOmMultipartKeyInfo().getPartKeyInfoMap();
+    for (Map.Entry<Integer, PartKeyInfo > partKeyInfoEntry :
+        partKeyInfoMap.entrySet()) {
+      PartKeyInfo partKeyInfo = partKeyInfoEntry.getValue();
+      OmKeyInfo currentKeyPartInfo =
+          OmKeyInfo.getFromProtobuf(partKeyInfo.getPartKeyInfo());
+
+      RepeatedOmKeyInfo repeatedOmKeyInfo =
+          omMetadataManager.getDeletedTable().get(partKeyInfo.getPartName());

Review comment:
       Thanks again for the detailed explanation. Good catch!
   
   I have updated the partName logic and the partName format is now `<parentID>/fileName + ClientID`.
   
   Since the `partKeyInfo.getPartName()` stored in DB is in the above format all the DB operations will happen using this dbPartName. So, no specific conversion required in response class. 
   
   To the user after commit, it will return the` fullKeyPartName` . [Reference code:](https://github.com/apache/ozone/blob/HDDS-2939/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java#L225).




----------------------------------------------------------------
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 #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/multipart/S3MultipartUploadAbortResponseV1.java
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.response.s3.multipart;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.MULTIPARTFILEINFO_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
+
+/**
+ * Response for Multipart Abort Request layout version V1.
+ */
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, DELETED_TABLE,
+    MULTIPARTFILEINFO_TABLE})
+public class S3MultipartUploadAbortResponseV1
+    extends S3MultipartUploadAbortResponse {
+
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse,
+      String multipartKey, @Nonnull OmMultipartKeyInfo omMultipartKeyInfo,
+      boolean isRatisEnabled, @Nonnull OmBucketInfo omBucketInfo) {
+
+    super(omResponse, multipartKey, omMultipartKeyInfo, isRatisEnabled,
+        omBucketInfo);
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // Delete from openKey table and multipart info table.
+    omMetadataManager.getOpenKeyTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+    omMetadataManager.getMultipartInfoTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+
+    // Move all the parts to delete table
+    TreeMap<Integer, PartKeyInfo > partKeyInfoMap =
+        getOmMultipartKeyInfo().getPartKeyInfoMap();
+    for (Map.Entry<Integer, PartKeyInfo > partKeyInfoEntry :
+        partKeyInfoMap.entrySet()) {
+      PartKeyInfo partKeyInfo = partKeyInfoEntry.getValue();
+      OmKeyInfo currentKeyPartInfo =
+          OmKeyInfo.getFromProtobuf(partKeyInfo.getPartKeyInfo());
+
+      RepeatedOmKeyInfo repeatedOmKeyInfo =
+          omMetadataManager.getDeletedTable().get(partKeyInfo.getPartName());

Review comment:
       Thanks for addressing the comment, @rakeshadr .
   The reference UT verified that the part key entries are correctly put into the delete table.
   But the key name here might be a full part key name I think, as delete table format is like below:
   
   ```java
   |----------------------------------------------------------------------|
   | deletedTable       | /volumeName/bucketName/keyName->RepeatedKeyInfo |
   |----------------------------------------------------------------------|
   ```
   
   Now part name stored in omMultipartKeyInfo is filename+clientID, use this part name as a key name in delete table doesn't match above format.
   
   Correct me If I am wrong.




----------------------------------------------------------------
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 #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/multipart/S3MultipartUploadAbortResponseV1.java
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.response.s3.multipart;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.MULTIPARTFILEINFO_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
+
+/**
+ * Response for Multipart Abort Request layout version V1.
+ */
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, DELETED_TABLE,
+    MULTIPARTFILEINFO_TABLE})
+public class S3MultipartUploadAbortResponseV1
+    extends S3MultipartUploadAbortResponse {
+
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse,
+      String multipartKey, @Nonnull OmMultipartKeyInfo omMultipartKeyInfo,
+      boolean isRatisEnabled, @Nonnull OmBucketInfo omBucketInfo) {
+
+    super(omResponse, multipartKey, omMultipartKeyInfo, isRatisEnabled,
+        omBucketInfo);
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // Delete from openKey table and multipart info table.
+    omMetadataManager.getOpenKeyTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+    omMetadataManager.getMultipartInfoTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+
+    // Move all the parts to delete table
+    TreeMap<Integer, PartKeyInfo > partKeyInfoMap =
+        getOmMultipartKeyInfo().getPartKeyInfoMap();
+    for (Map.Entry<Integer, PartKeyInfo > partKeyInfoEntry :
+        partKeyInfoMap.entrySet()) {
+      PartKeyInfo partKeyInfo = partKeyInfoEntry.getValue();
+      OmKeyInfo currentKeyPartInfo =
+          OmKeyInfo.getFromProtobuf(partKeyInfo.getPartKeyInfo());
+
+      RepeatedOmKeyInfo repeatedOmKeyInfo =
+          omMetadataManager.getDeletedTable().get(partKeyInfo.getPartName());

Review comment:
       Seems here addToDBBatch logic can completely reuse S3MultipartUploadAbortResponse#addToDBBatch. 
   The only difference is that: In V1 layout, partName stored in part keyInfo is not a full part key name (now is filename+clientID, [link](https://github.com/apache/ozone/blob/HDDS-2939/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java#L162)), but we should reconstruct the full part key name to find in delete table.
   Can we have the corresponding test case cover this?




----------------------------------------------------------------
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 #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/multipart/S3MultipartUploadAbortResponseV1.java
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.response.s3.multipart;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.MULTIPARTFILEINFO_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
+
+/**
+ * Response for Multipart Abort Request layout version V1.
+ */
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, DELETED_TABLE,
+    MULTIPARTFILEINFO_TABLE})
+public class S3MultipartUploadAbortResponseV1
+    extends S3MultipartUploadAbortResponse {
+
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse,
+      String multipartKey, @Nonnull OmMultipartKeyInfo omMultipartKeyInfo,
+      boolean isRatisEnabled, @Nonnull OmBucketInfo omBucketInfo) {
+
+    super(omResponse, multipartKey, omMultipartKeyInfo, isRatisEnabled,
+        omBucketInfo);
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // Delete from openKey table and multipart info table.
+    omMetadataManager.getOpenKeyTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+    omMetadataManager.getMultipartInfoTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+
+    // Move all the parts to delete table
+    TreeMap<Integer, PartKeyInfo > partKeyInfoMap =
+        getOmMultipartKeyInfo().getPartKeyInfoMap();
+    for (Map.Entry<Integer, PartKeyInfo > partKeyInfoEntry :
+        partKeyInfoMap.entrySet()) {
+      PartKeyInfo partKeyInfo = partKeyInfoEntry.getValue();
+      OmKeyInfo currentKeyPartInfo =
+          OmKeyInfo.getFromProtobuf(partKeyInfo.getPartKeyInfo());
+
+      RepeatedOmKeyInfo repeatedOmKeyInfo =
+          omMetadataManager.getDeletedTable().get(partKeyInfo.getPartName());

Review comment:
       Seems here addToDBBatch logic can completely reuse S3MultipartUploadAbortResponse#addToDBBatch. 
   The only difference is that: In V1 layout, partName stored in part keyInfo is not a full part key name (now is filename+clientID, [link](https://github.com/apache/ozone/blob/HDDS-2939/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequestV1.java#L162)), but we should reconstruct the full part key name to find in delete table.




----------------------------------------------------------------
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 #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/multipart/S3MultipartUploadAbortResponseV1.java
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.response.s3.multipart;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.MULTIPARTFILEINFO_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
+
+/**
+ * Response for Multipart Abort Request layout version V1.
+ */
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, DELETED_TABLE,
+    MULTIPARTFILEINFO_TABLE})
+public class S3MultipartUploadAbortResponseV1
+    extends S3MultipartUploadAbortResponse {
+
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse,
+      String multipartKey, @Nonnull OmMultipartKeyInfo omMultipartKeyInfo,
+      boolean isRatisEnabled, @Nonnull OmBucketInfo omBucketInfo) {
+
+    super(omResponse, multipartKey, omMultipartKeyInfo, isRatisEnabled,
+        omBucketInfo);
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public S3MultipartUploadAbortResponseV1(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // Delete from openKey table and multipart info table.
+    omMetadataManager.getOpenKeyTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+    omMetadataManager.getMultipartInfoTable().deleteWithBatch(batchOperation,
+        getMultipartKey());
+
+    // Move all the parts to delete table
+    TreeMap<Integer, PartKeyInfo > partKeyInfoMap =
+        getOmMultipartKeyInfo().getPartKeyInfoMap();
+    for (Map.Entry<Integer, PartKeyInfo > partKeyInfoEntry :
+        partKeyInfoMap.entrySet()) {
+      PartKeyInfo partKeyInfo = partKeyInfoEntry.getValue();
+      OmKeyInfo currentKeyPartInfo =
+          OmKeyInfo.getFromProtobuf(partKeyInfo.getPartKeyInfo());
+
+      RepeatedOmKeyInfo repeatedOmKeyInfo =
+          omMetadataManager.getDeletedTable().get(partKeyInfo.getPartName());

Review comment:
       Thanks a lot @linyiqun for the reviews. 
   
   Very good point. I have made the changes. I've made few more assertions for the upload parts.
   
   For the deleted table part entries, there is a unit test case and this patch extending TestS3MultipartUploadAbortResponseV1#testAddDBToBatchWithParts. Hope thats fine for you. [UT Reference](https://github.com/apache/ozone/blob/master/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3MultipartUploadAbortResponse.java#L133)




----------------------------------------------------------------
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 #1997: HDDS-4835. [FSO]S3Multipart: Implement UploadAbortRequest

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


   


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