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 2022/09/26 04:05:34 UTC

[GitHub] [ozone] Xushaohong opened a new pull request, #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Xushaohong opened a new pull request, #3778:
URL: https://github.com/apache/ozone/pull/3778

   ## What changes were proposed in this pull request?
   
   We found that the blocks the client committed may not be equal to the allocated(when EC stripe fails), and the uncommitted blocks would remain forever as orphan blocks as no key maps to them anymore. 
   
   Here I use a tricky solution that considers these blocks as repeated/ overwritten key blocks, and thus they would be deleted in the existing deletion path.
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-7258
   
   ## How was this patch tested?
   UT and Manual test in the test environment.
   


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1001327157


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java:
##########
@@ -177,6 +181,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
         omBucketInfo.incrUsedNamespace(1L);
       }
 
+      // let the uncommitted blocks pretend as key's old version blocks
+      // which will be deleted as RepeatedOmKeyInfo
+      if (!uncommitted.isEmpty()) {
+        LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
+            uncommitted, keyName);
+        OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+        // set dataSize -1 here to distinguish from normal keyInfo
+        pseudoKeyInfo.setDataSize(-1);
+        List<OmKeyLocationInfoGroup> uncommittedGroups = new ArrayList<>();
+        // version not matters in the current logic of keyDeletingService,
+        // all versions of blocks will be deleted.
+        uncommittedGroups.add(new OmKeyLocationInfoGroup(0, uncommitted));
+        pseudoKeyInfo.setKeyLocationVersions(uncommittedGroups);
+        if (oldKeyVersionsToDelete == null) {
+          oldKeyVersionsToDelete = new RepeatedOmKeyInfo(pseudoKeyInfo);
+        } else {
+          oldKeyVersionsToDelete.addOmKeyInfo(pseudoKeyInfo);
+        }
+      }

Review Comment:
   Good idea! Updated! 



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] kaijchen commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
kaijchen commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1001869852


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java:
##########
@@ -239,6 +243,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
         omBucketInfo.incrUsedNamespace(1L);
       }
 
+      // let the uncommitted blocks pretend as key's old version blocks
+      // which will be deleted as RepeatedOmKeyInfo
+      if (!uncommitted.isEmpty()) {
+        LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
+            uncommitted, keyName);
+        OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+        // set dataSize -1 here to distinguish from normal keyInfo
+        pseudoKeyInfo.setDataSize(-1);

Review Comment:
   Can we add a field such as `boolean persudo` to OmKeyInfo instead?



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] kaijchen commented on pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
kaijchen commented on PR #3778:
URL: https://github.com/apache/ozone/pull/3778#issuecomment-1272244304

   Thanks for the work @Xushaohong.
   Is it possible to delete those blocks directly instead of relying on OpenKeyCleanupService?
   I'm worrying about too much coupling here.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r990605708


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -214,17 +219,29 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
       updatedBlockLocations =
           verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
     }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+
+    // Every time the key commits, the uncommitted blocks should be detected
+    // If client not commit, the blocks should be cleaned by Open Key CleanUp
+    // Service.
+    // keyLocationInfoGroup has the allocated block location info
+    List<OmKeyLocationInfo> uncommittedBlocks =
+        new ArrayList<>(keyLocationInfoGroup.getBlocksLatestVersionOnly());
+    // Only check ContainerBlockID here to avoid the mismatch of the pipeline
+    // field and BcsId in the OmKeyLocationInfo, as the OmKeyInfoCodec ignores
+    // the pipeline field by default and bcsId would be updated in Ratis mode.
+    List<ContainerBlockID> updatedBlockIDs = updatedBlockLocations.stream().

Review Comment:
   > It's better to use `Set<ContainerBlockID> updatedBlockIDs` here.
   
   Thx for the suggestion, but it should be fine here.
   1. The duplication of BlockIDs is not harmful since it will be only used for checking if two lists have intersections.
   2. Committing the duplicate blocks in one request could nearly happen and in **verifyAndGetKeyLocations** the duplicate should be already 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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] sodonnel commented on pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
sodonnel commented on PR #3778:
URL: https://github.com/apache/ozone/pull/3778#issuecomment-1258351585

   Could you please describe how these blocks are being detected in the PR description - like a small design overview?


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r990604640


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -214,17 +219,29 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
       updatedBlockLocations =
           verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
     }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+
+    // Every time the key commits, the uncommitted blocks should be detected
+    // If client not commit, the blocks should be cleaned by Open Key CleanUp
+    // Service.
+    // keyLocationInfoGroup has the allocated block location info
+    List<OmKeyLocationInfo> uncommittedBlocks =
+        new ArrayList<>(keyLocationInfoGroup.getBlocksLatestVersionOnly());
+    // Only check ContainerBlockID here to avoid the mismatch of the pipeline
+    // field and BcsId in the OmKeyLocationInfo, as the OmKeyInfoCodec ignores
+    // the pipeline field by default and bcsId would be updated in Ratis mode.
+    List<ContainerBlockID> updatedBlockIDs = updatedBlockLocations.stream().
+        map(BlockLocationInfo::getBlockID).map(BlockID::getContainerBlockID).
+        collect(Collectors.toList());
+    uncommittedBlocks.removeIf(block -> updatedBlockIDs.

Review Comment:
   List<ContainerBlockID> updatedBlockIDs 
   List<OmKeyLocationInfo> uncommittedBlocks 
   
   They have two kinds of element types.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] kaijchen commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
kaijchen commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r990606405


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -214,17 +219,29 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
       updatedBlockLocations =
           verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
     }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+
+    // Every time the key commits, the uncommitted blocks should be detected
+    // If client not commit, the blocks should be cleaned by Open Key CleanUp
+    // Service.
+    // keyLocationInfoGroup has the allocated block location info
+    List<OmKeyLocationInfo> uncommittedBlocks =
+        new ArrayList<>(keyLocationInfoGroup.getBlocksLatestVersionOnly());
+    // Only check ContainerBlockID here to avoid the mismatch of the pipeline
+    // field and BcsId in the OmKeyLocationInfo, as the OmKeyInfoCodec ignores
+    // the pipeline field by default and bcsId would be updated in Ratis mode.
+    List<ContainerBlockID> updatedBlockIDs = updatedBlockLocations.stream().

Review Comment:
   `Set.contains()` is faster if block count is large.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3778:
URL: https://github.com/apache/ozone/pull/3778#issuecomment-1259418406

   https://docs.google.com/document/d/1Oi1zPKdmvA7DFwIcUWz6zw_p-pY3D1L76gyV37jQT94/edit# 
   
   The prototype of the design. 


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3778:
URL: https://github.com/apache/ozone/pull/3778#issuecomment-1272246324

   > 
   Thx for the question~ @kaijchen 
   The case for OpenKeyCleanupService is not related to the logic here. They should be two cases as I mentioned in the DOC.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] errose28 commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
errose28 commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r998689250


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java:
##########
@@ -190,6 +191,93 @@ public void testValidateAndUpdateCache() throws Exception {
         omKeyInfo.getLatestVersionLocations().getLocationList());
   }
 
+  @Test
+  public void testValidateAndUpdateCacheWithUncommittedBlocks()
+      throws Exception {
+
+    // allocated block list
+    List<KeyLocation> allocatedKeyLocationList = getKeyLocation(5, 0);
+
+    List<OmKeyLocationInfo> allocatedBlockList = allocatedKeyLocationList
+        .stream().map(OmKeyLocationInfo::getFromProtobuf)
+        .collect(Collectors.toList());
+
+    // committed block list, with three blocks different with the allocated
+    List<KeyLocation> committedKeyLocationList = getKeyLocation(5, 3);

Review Comment:
   It looks like the purpose of this new `getKeyLocation(int, int)` method is to create block lists with some overlap. However the client should never be able to commit blocks that were not already allocated in the open key. See #2108, which modified `OMKeyInfo#verifyAndGetKeyLocations` to drop the extra blocks and added a corresponding unit test in this class. Therefore in this test I think we only need to test the client committing a subset of the blocks it allocated. We could use the original `getKeyLocation(int)` method to make the allocated list, copy this, and just remove a block or two to create the commit list without needing a new method.



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -214,17 +220,29 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
       updatedBlockLocations =
           verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
     }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+
+    // Every time the key commits, the uncommitted blocks should be detected
+    // If client not commit, the blocks should be cleaned by Open Key CleanUp
+    // Service.
+    // keyLocationInfoGroup has the allocated block location info
+    List<OmKeyLocationInfo> uncommittedBlocks =
+        new ArrayList<>(keyLocationInfoGroup.getBlocksLatestVersionOnly());
+    // Only check ContainerBlockID here to avoid the mismatch of the pipeline
+    // field and BcsId in the OmKeyLocationInfo, as the OmKeyInfoCodec ignores
+    // the pipeline field by default and bcsId would be updated in Ratis mode.
+    Set<ContainerBlockID> updatedBlockIDs = updatedBlockLocations.stream().
+        map(BlockLocationInfo::getBlockID).map(BlockID::getContainerBlockID).
+        collect(Collectors.toSet());
+    uncommittedBlocks.removeIf(block -> updatedBlockIDs.
+        contains(block.getBlockID().getContainerBlockID()));
+

Review Comment:
   The allocated block list has already been iterated once in the call to `verifyAndGetKeyLocations` above. Can we refactor this so that the checks in that method and these new checks can be done in one iteration?
   
   In total we need to identify blocks that are in the allocated list but not in the commit list, and blocks that are in the commit list but not in the allocated list. We could
   1. Convert the allocated list to a set.
   2. Iterate the commit list.
       - if the current block in the commit list is not in the allocated set, drop it and log a warning like the code currently does.
       - else remove the current block in the commit list from the allocated set.
    3. The blocks remaining in the allocated set can be returned as the uncommitted blocks to be deleted.
    4. The blocks left in the commit list are ok to be committed.
   
   This code is on the main write path, so we should try to make it as efficient as possible.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] kaijchen commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
kaijchen commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1001329591


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -207,50 +211,65 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
 
     // Compare user given block location against allocatedBlockLocations
     // present in OmKeyInfo.
+    List<OmKeyLocationInfo> uncommittedBlocks = new ArrayList<>();
     List<OmKeyLocationInfo> updatedBlockLocations;
     if (skipBlockIDCheck) {
       updatedBlockLocations = locationInfoList;
     } else {
-      updatedBlockLocations =
-          verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
-    }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+      updatedBlockLocations = verifyAndGetKeyLocations(locationInfoList,
+          keyLocationInfoGroup, uncommittedBlocks);
+    }
+
     keyLocationInfoGroup.removeBlocks(latestVersion);
     // set each of the locationInfo object to the latest version
     updatedBlockLocations.forEach(omKeyLocationInfo -> omKeyLocationInfo
         .setCreateVersion(latestVersion));
     keyLocationInfoGroup.addAll(latestVersion, updatedBlockLocations);
+
+    return uncommittedBlocks;
   }
 
+  /**
+   *  1. Verify committed KeyLocationInfos
+   *  2. find out the allocated but uncommitted KeyLocationInfos.
+   *
+   * @param locationInfoList committed KeyLocationInfos
+   * @param keyLocationInfoGroup allocated KeyLocationInfoGroup
+   * @param uncommittedLocationInfos list of uncommitted KeyLocationInfos

Review Comment:
   Please remove this parameter since it is returned in `Pair<updatedBlockLocations, uncommittedBlocks>`.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3778:
URL: https://github.com/apache/ozone/pull/3778#issuecomment-1291389007

   @kaijchen @errose28 Thx for the review! :)


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1000291110


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -214,17 +220,29 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
       updatedBlockLocations =
           verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
     }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+
+    // Every time the key commits, the uncommitted blocks should be detected
+    // If client not commit, the blocks should be cleaned by Open Key CleanUp
+    // Service.
+    // keyLocationInfoGroup has the allocated block location info
+    List<OmKeyLocationInfo> uncommittedBlocks =
+        new ArrayList<>(keyLocationInfoGroup.getBlocksLatestVersionOnly());
+    // Only check ContainerBlockID here to avoid the mismatch of the pipeline
+    // field and BcsId in the OmKeyLocationInfo, as the OmKeyInfoCodec ignores
+    // the pipeline field by default and bcsId would be updated in Ratis mode.
+    Set<ContainerBlockID> updatedBlockIDs = updatedBlockLocations.stream().
+        map(BlockLocationInfo::getBlockID).map(BlockID::getContainerBlockID).
+        collect(Collectors.toSet());
+    uncommittedBlocks.removeIf(block -> updatedBlockIDs.
+        contains(block.getBlockID().getContainerBlockID()));
+

Review Comment:
   The point that saving extra iteration and doing it in **verifyAndGetKeyLocations** is a good idea!
   We need to compare two lists, but the key to distinguishing is the **ContainerBlockID** field.
   So we should not simply convert the allocated list to a set to avoid iterating the allocated list each time when checking if the **ContainerBlockID** matches.
   Now I use the HashMap to store the ContainerBlockID as the key and thus save the time complexity. The cost is relatively small and removes the extra iteration of allocated block.
    
   



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] errose28 commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
errose28 commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1001120819


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java:
##########
@@ -177,6 +181,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
         omBucketInfo.incrUsedNamespace(1L);
       }
 
+      // let the uncommitted blocks pretend as key's old version blocks
+      // which will be deleted as RepeatedOmKeyInfo
+      if (!uncommitted.isEmpty()) {
+        LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
+            uncommitted, keyName);
+        OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+        // set dataSize -1 here to distinguish from normal keyInfo
+        pseudoKeyInfo.setDataSize(-1);
+        List<OmKeyLocationInfoGroup> uncommittedGroups = new ArrayList<>();
+        // version not matters in the current logic of keyDeletingService,
+        // all versions of blocks will be deleted.
+        uncommittedGroups.add(new OmKeyLocationInfoGroup(0, uncommitted));
+        pseudoKeyInfo.setKeyLocationVersions(uncommittedGroups);
+        if (oldKeyVersionsToDelete == null) {
+          oldKeyVersionsToDelete = new RepeatedOmKeyInfo(pseudoKeyInfo);
+        } else {
+          oldKeyVersionsToDelete.addOmKeyInfo(pseudoKeyInfo);
+        }
+      }

Review Comment:
   Can we move this duplicated code block used by the FSO and OBS versions of this request to a helper method in the parent OMKeyRequest class?



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java:
##########
@@ -239,6 +243,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
         omBucketInfo.incrUsedNamespace(1L);
       }
 
+      // let the uncommitted blocks pretend as key's old version blocks
+      // which will be deleted as RepeatedOmKeyInfo
+      if (!uncommitted.isEmpty()) {
+        LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
+            uncommitted, keyName);
+        OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+        // set dataSize -1 here to distinguish from normal keyInfo
+        pseudoKeyInfo.setDataSize(-1);

Review Comment:
   Why is this necessary? Can we not set this value instead?



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1001326570


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java:
##########
@@ -239,6 +243,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
         omBucketInfo.incrUsedNamespace(1L);
       }
 
+      // let the uncommitted blocks pretend as key's old version blocks
+      // which will be deleted as RepeatedOmKeyInfo
+      if (!uncommitted.isEmpty()) {
+        LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
+            uncommitted, keyName);
+        OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+        // set dataSize -1 here to distinguish from normal keyInfo
+        pseudoKeyInfo.setDataSize(-1);

Review Comment:
   Setting -1 here is to have an explicit difference with normal keyInfo.
   1. The pseudoKeyInfo is copied from the keyInfo, so the data size comes from the client and is the real key length, which is irrelevant to these uncommitted blocks. If we don't reset this, it may get confusing with **OmKeyInfo** which has real key block info.
   2. It could be a debug trace.
   So I think it is good to have~



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java:
##########
@@ -239,6 +243,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
         omBucketInfo.incrUsedNamespace(1L);
       }
 
+      // let the uncommitted blocks pretend as key's old version blocks
+      // which will be deleted as RepeatedOmKeyInfo
+      if (!uncommitted.isEmpty()) {
+        LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
+            uncommitted, keyName);
+        OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+        // set dataSize -1 here to distinguish from normal keyInfo
+        pseudoKeyInfo.setDataSize(-1);

Review Comment:
   Setting -1 here is to have an explicit difference with normal keyInfo.
   1. The pseudoKeyInfo is copied from the keyInfo, so the data size comes from the client and is the real key length, which is irrelevant to these uncommitted blocks. If we don't reset this, it may get confusing with **OmKeyInfo** which has real key block info.
   2. It could be a debug trace.
   
   So I think it is good to have~



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1001327411


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -207,50 +211,65 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
 
     // Compare user given block location against allocatedBlockLocations
     // present in OmKeyInfo.
+    List<OmKeyLocationInfo> uncommittedBlocks = new ArrayList<>();
     List<OmKeyLocationInfo> updatedBlockLocations;
     if (skipBlockIDCheck) {
       updatedBlockLocations = locationInfoList;
     } else {
-      updatedBlockLocations =
-          verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
-    }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+      updatedBlockLocations = verifyAndGetKeyLocations(locationInfoList,
+          keyLocationInfoGroup, uncommittedBlocks);
+    }
+
     keyLocationInfoGroup.removeBlocks(latestVersion);
     // set each of the locationInfo object to the latest version
     updatedBlockLocations.forEach(omKeyLocationInfo -> omKeyLocationInfo
         .setCreateVersion(latestVersion));
     keyLocationInfoGroup.addAll(latestVersion, updatedBlockLocations);
+
+    return uncommittedBlocks;
   }
 
+  /**
+   *  1. Verify committed KeyLocationInfos
+   *  2. find out the allocated but uncommitted KeyLocationInfos.
+   *
+   * @param locationInfoList committed KeyLocationInfos
+   * @param keyLocationInfoGroup allocated KeyLocationInfoGroup
+   * @param uncommittedLocationInfos list of uncommitted KeyLocationInfos

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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1000270745


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java:
##########
@@ -190,6 +191,93 @@ public void testValidateAndUpdateCache() throws Exception {
         omKeyInfo.getLatestVersionLocations().getLocationList());
   }
 
+  @Test
+  public void testValidateAndUpdateCacheWithUncommittedBlocks()
+      throws Exception {
+
+    // allocated block list
+    List<KeyLocation> allocatedKeyLocationList = getKeyLocation(5, 0);
+
+    List<OmKeyLocationInfo> allocatedBlockList = allocatedKeyLocationList
+        .stream().map(OmKeyLocationInfo::getFromProtobuf)
+        .collect(Collectors.toList());
+
+    // committed block list, with three blocks different with the allocated
+    List<KeyLocation> committedKeyLocationList = getKeyLocation(5, 3);

Review Comment:
   Thx @errose28  for the review! 
   Actually committing the unknown block from the client side is still possible, https://github.com/apache/ozone/pull/2108 this work is on the server side. 
   But I have done some improvements like what you said, PTAL~



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] kaijchen commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
kaijchen commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1000557610


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -207,50 +211,65 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
 
     // Compare user given block location against allocatedBlockLocations
     // present in OmKeyInfo.
+    List<OmKeyLocationInfo> uncommittedBlocks = new ArrayList<>();
     List<OmKeyLocationInfo> updatedBlockLocations;
     if (skipBlockIDCheck) {
       updatedBlockLocations = locationInfoList;
     } else {
-      updatedBlockLocations =
-          verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
-    }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+      updatedBlockLocations = verifyAndGetKeyLocations(locationInfoList,
+          keyLocationInfoGroup, uncommittedBlocks);
+    }
+
     keyLocationInfoGroup.removeBlocks(latestVersion);
     // set each of the locationInfo object to the latest version
     updatedBlockLocations.forEach(omKeyLocationInfo -> omKeyLocationInfo
         .setCreateVersion(latestVersion));
     keyLocationInfoGroup.addAll(latestVersion, updatedBlockLocations);
+
+    return uncommittedBlocks;
   }
 
+  /**
+   *  1. Verify committed KeyLocationInfos
+   *  2. find out the allocated but uncommitted KeyLocationInfos.
+   *
+   * @param locationInfoList committed KeyLocationInfos
+   * @param keyLocationInfoGroup allocated KeyLocationInfoGroup
+   * @param uncommittedLocationInfos list of uncommitted KeyLocationInfos

Review Comment:
   Javadoc for `uncommittedLocationInfos` should reflect it's a field being written to.
   For example `a List where the uncommitted KeyLocationInfos will be added into`.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] errose28 merged pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
errose28 merged PR #3778:
URL: https://github.com/apache/ozone/pull/3778


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1002852397


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java:
##########
@@ -239,6 +243,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
         omBucketInfo.incrUsedNamespace(1L);
       }
 
+      // let the uncommitted blocks pretend as key's old version blocks
+      // which will be deleted as RepeatedOmKeyInfo
+      if (!uncommitted.isEmpty()) {
+        LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
+            uncommitted, keyName);
+        OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+        // set dataSize -1 here to distinguish from normal keyInfo
+        pseudoKeyInfo.setDataSize(-1);

Review Comment:
   I removed the reset here. @errose28 @kaijchen 
   We could revisit it if necessary.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] kaijchen commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
kaijchen commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r990603014


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -214,17 +219,29 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
       updatedBlockLocations =
           verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
     }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+
+    // Every time the key commits, the uncommitted blocks should be detected
+    // If client not commit, the blocks should be cleaned by Open Key CleanUp
+    // Service.
+    // keyLocationInfoGroup has the allocated block location info
+    List<OmKeyLocationInfo> uncommittedBlocks =
+        new ArrayList<>(keyLocationInfoGroup.getBlocksLatestVersionOnly());
+    // Only check ContainerBlockID here to avoid the mismatch of the pipeline
+    // field and BcsId in the OmKeyLocationInfo, as the OmKeyInfoCodec ignores
+    // the pipeline field by default and bcsId would be updated in Ratis mode.
+    List<ContainerBlockID> updatedBlockIDs = updatedBlockLocations.stream().

Review Comment:
   It's better to use `Set<ContainerBlockID> updatedBlockIDs` here.
   



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java:
##########
@@ -214,17 +219,29 @@ public void updateLocationInfoList(List<OmKeyLocationInfo> locationInfoList,
       updatedBlockLocations =
           verifyAndGetKeyLocations(locationInfoList, keyLocationInfoGroup);
     }
-    // Updates the latest locationList in the latest version only with
-    // given locationInfoList here.
-    // TODO : The original allocated list and the updated list here may vary
-    // as the containers on the Datanode on which the blocks were pre allocated
-    // might get closed. The diff of blocks between these two lists here
-    // need to be garbage collected in case the ozone client dies.
+
+    // Every time the key commits, the uncommitted blocks should be detected
+    // If client not commit, the blocks should be cleaned by Open Key CleanUp
+    // Service.
+    // keyLocationInfoGroup has the allocated block location info
+    List<OmKeyLocationInfo> uncommittedBlocks =
+        new ArrayList<>(keyLocationInfoGroup.getBlocksLatestVersionOnly());
+    // Only check ContainerBlockID here to avoid the mismatch of the pipeline
+    // field and BcsId in the OmKeyLocationInfo, as the OmKeyInfoCodec ignores
+    // the pipeline field by default and bcsId would be updated in Ratis mode.
+    List<ContainerBlockID> updatedBlockIDs = updatedBlockLocations.stream().
+        map(BlockLocationInfo::getBlockID).map(BlockID::getContainerBlockID).
+        collect(Collectors.toList());
+    uncommittedBlocks.removeIf(block -> updatedBlockIDs.

Review Comment:
   Can be simplified to `uncommittedBlocks.removeAll(updatedBlockIDs);`
   
   `uncommittedBlocks` could also be a `Set` if order doesn't matter.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3778:
URL: https://github.com/apache/ozone/pull/3778#issuecomment-1259192689

   > Could you please describe how these blocks are being detected in the PR description - like a small design overview?
   
   Sure, will update later, I am working on fixing some issues found by UT.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3778:
URL: https://github.com/apache/ozone/pull/3778#issuecomment-1282234842

   Hi, all!  I have done the refactor of this PR. Could you take a look ~


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] kaijchen commented on pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
kaijchen commented on PR #3778:
URL: https://github.com/apache/ozone/pull/3778#issuecomment-1285479747

   Thanks @Xushaohong for updating the patch. Filtering uncommitted blocks in `verifyAndGetKeyLocations` is a good idea. For clearity, it's better to return a `Pair<updatedBlockLocations, uncommittedBlocks>` instead, or create a inner 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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] errose28 commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
errose28 commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1003866905


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java:
##########
@@ -816,4 +816,28 @@ protected OzoneLockStrategy getOzoneLockStrategy(OzoneManager ozoneManager) {
     return ozoneManager.getOzoneLockProvider()
         .createLockStrategy(getBucketLayout());
   }
+
+  /**
+   * Warp the uncommitted blocks as pseudoKeyInfo.
+   *
+   * @param uncommitted Uncommitted OmKeyLocationInfo
+   * @param omKeyInfo   Args for key block
+   * @return pseudoKeyInfo
+   */
+  protected OmKeyInfo warpUncommittedBlocksAsPseudoKey(

Review Comment:
   Do you mean wrap instead of warp in the method name and javadoc?



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] Xushaohong commented on a diff in pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on code in PR #3778:
URL: https://github.com/apache/ozone/pull/3778#discussion_r1003924979


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java:
##########
@@ -816,4 +816,28 @@ protected OzoneLockStrategy getOzoneLockStrategy(OzoneManager ozoneManager) {
     return ozoneManager.getOzoneLockProvider()
         .createLockStrategy(getBucketLayout());
   }
+
+  /**
+   * Warp the uncommitted blocks as pseudoKeyInfo.
+   *
+   * @param uncommitted Uncommitted OmKeyLocationInfo
+   * @param omKeyInfo   Args for key block
+   * @return pseudoKeyInfo
+   */
+  protected OmKeyInfo warpUncommittedBlocksAsPseudoKey(

Review Comment:
   Oops, this is a spelling error. Fixed!



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] kaijchen commented on pull request #3778: HDDS-7258. Cleanup the allocated but uncommitted blocks

Posted by GitBox <gi...@apache.org>.
kaijchen commented on PR #3778:
URL: https://github.com/apache/ozone/pull/3778#issuecomment-1272247007

   > Thx for the question~ @kaijchen
   > The case for OpenKeyCleanupService is not related to the logic here. They should be two cases as I mentioned in the DOC.
   
   Sorry I misread your purposal. The pseudoKeyInfo is moved into DeleteKeyTable
   instead of staying at OpenKeyTable, which sounds reasonable 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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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