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/07/19 12:06:03 UTC

[GitHub] [ozone] kuenishi opened a new pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

kuenishi opened a new pull request #2433:
URL: https://github.com/apache/ozone/pull/2433


   ## What changes were proposed in this pull request?
   
   HDDS-5243 was a patch for omitting key locations for clients on reading. But the same warning of large response size observed in our cluster for putting data. This is harmful in terms of retry storm, as hadoop-rpc handles this large-response-exception as retry-able exception. Thus, RetryInvocationHandler retries, despite it cannot be recovered by retry, for 15 times, receiving large response message exceeding default limit of RPC message size 128MB as follows.
   
   ```
       2021-06-21 19:23:10,717 [IPC Server handler 65 on default port 9862] WARN org.apache.hadoop.ipc.Server: Large response size 134538349 for call Call#2037538 Retry#15 org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol.submitRequest from 10.192.17.172:34070
       2021-06-21 19:23:10,722 [IPC Server handler 65 on default port 9862] WARN org.apache.hadoop.ipc.Server: IPC Server handler 65 on default port 9862, call Call#2037538 Retry#15 org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol.submitRequest from 10.192.17.172:34070: output error
       2021-06-21 19:23:10,722 [IPC Server handler 65 on default port 9862] INFO org.apache.hadoop.ipc.Server: IPC Server handler 65 on default port 9862 caught an exception
       java.nio.channels.AsynchronousCloseException
       at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:205)
       at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:478)
       at org.apache.hadoop.ipc.Server.channelIO(Server.java:3642)
       at org.apache.hadoop.ipc.Server.channelWrite(Server.java:3594)
       at org.apache.hadoop.ipc.Server.access$1700(Server.java:139)
       at org.apache.hadoop.ipc.Server$Responder.processResponse(Server.java:1657)
       at org.apache.hadoop.ipc.Server$Responder.doRespond(Server.java:1727)
   ```
   
   Suggestion in HDDS-5393 was wrong and it shall be fixed by making old blocks eligible for deletion service, moving to deletion table. It is only needed for normal object-put, while not needed for MultipartUpload objects, if I understand correctly. 
   
   ## What is the link to the Apache JIRA
   
   [HDDS-5461](https://issues.apache.org/jira/browse/HDDS-5461)
   
   Please replace this section with the link to the Apache JIRA)
   
   ## How was this patch tested?
   
   - Add tiny unit test that checks to `TestOMKeyCommitResponse`
   - Manual & local test using AWS CLI with debug log
   


-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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






-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java
##########
@@ -29,28 +30,31 @@
 import java.io.IOException;
 import javax.annotation.Nonnull;
 
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_KEY_TABLE;
 
 /**
  * Response for CommitKey request.
  */
-@CleanupTableInfo(cleanupTables = {OPEN_KEY_TABLE, KEY_TABLE})
+@CleanupTableInfo(cleanupTables = {OPEN_KEY_TABLE, KEY_TABLE, DELETED_TABLE})

Review comment:
       Now using cache.




-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -239,6 +259,40 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
     return omClientResponse;
   }
 
+  /**
+   * Prepare key for deletion service on overwrite.
+   *
+   * @param dbOzoneKey key to point to an object in RocksDB
+   * @param omMetadataManager
+   * @param isVersionEnabled
+   * @param trxnLogIndex
+   * @param isRatisEnabled
+   * @return Old keys eligible for deletion.
+   * @throws IOException
+   */
+  protected RepeatedOmKeyInfo getOldVersionsToCleanUp(
+          String dbOzoneKey, OMMetadataManager omMetadataManager,
+          boolean isVersionEnabled, long trxnLogIndex,
+          boolean isRatisEnabled) throws IOException {
+    if (isVersionEnabled) {
+      // Nothing to clean up in case versioning is on.
+      return null;
+    }
+    // Past keys that was deleted but still in deleted table,
+    // waiting for deletion service.
+    RepeatedOmKeyInfo keysToDelete =
+            omMetadataManager.getDeletedTable().getReadCopy(dbOzoneKey);

Review comment:
       I've changed to use table cache, so I'll keep this line.




-- 
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] bharatviswa504 merged pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   


-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   I have left the comments open which needs to be addressed to make it easy. 


-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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






-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   Test failure TestSCMInstallSnapshotWithHA looks unrelated.


-- 
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] adoroszlai commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   @bharatviswa504 please take a look at the most recent changes


-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
##########
@@ -286,8 +287,17 @@ public synchronized long addNewVersion(
       // it is important that the new version are always at the tail of the list
       OmKeyLocationInfoGroup currentLatestVersion =
           keyLocationVersions.get(keyLocationVersions.size() - 1);
+
+      // The new version is created based on the latest version number
+      // and will not include key locations of old versions, until object

Review comment:
       Removed the wrong comment.




-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   @ChenSammi @bharatviswa504 I have added option 3 to @symious 's [document](https://docs.google.com/document/d/1UYVETpSHyVppNPjHq8VuXj2t2Y0LYQNRYa6awSTCROA/edit#heading=h.h2k2ovrsrw33) and implemented it in this pull request. Multipart-uploaded objects shall be done in HDDS-5656 to keep this review smaller. 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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java
##########
@@ -94,4 +100,12 @@ protected OmBucketInfo getOmBucketInfo() {
   protected String getOzoneKeyName() {
     return ozoneKeyName;
   }
+
+  protected void maybeUpdateDeletedTable(OMMetadataManager omMetadataManager,

Review comment:
       Renamed




-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java
##########
@@ -275,6 +276,76 @@ public void testValidateAndUpdateCacheWithKeyNotFound() throws Exception {
     Assert.assertNull(omKeyInfo);
   }
 
+  @Test
+  public void testValidateAndUpdateCacheOnOverwrite() throws Exception {
+    testValidateAndUpdateCache();
+
+    // Become a new client and set next version number
+    clientID = Time.now();
+    version += 1;
+
+    OMRequest modifiedOmRequest = doPreExecute(createCommitKeyRequest());
+
+    OMKeyCommitRequest omKeyCommitRequest =
+            getOmKeyCommitRequest(modifiedOmRequest);
+
+    KeyArgs keyArgs = modifiedOmRequest.getCommitKeyRequest().getKeyArgs();
+
+    // Key should be there in key table, as validateAndUpdateCache is called.
+    OmKeyInfo omKeyInfo =
+        omMetadataManager.getKeyTable().get(getOzonePathKey());
+
+    Assert.assertNotNull(omKeyInfo);
+    // Previously committed version
+    Assert.assertEquals(0L,
+            omKeyInfo.getLatestVersionLocations().getVersion());
+
+    // Append new blocks
+    List<OmKeyLocationInfo> allocatedLocationList =
+            keyArgs.getKeyLocationsList().stream()
+                    .map(OmKeyLocationInfo::getFromProtobuf)
+                    .collect(Collectors.toList());
+    addKeyToOpenKeyTable(allocatedLocationList);
+
+    OMClientResponse omClientResponse =
+            omKeyCommitRequest.validateAndUpdateCache(ozoneManager,
+                    102L, ozoneManagerDoubleBufferHelper);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+            omClientResponse.getOMResponse().getStatus());
+
+    String ozoneKey = getOzonePathKey();
+    // Entry should be deleted from openKey Table.

Review comment:
       I merged your fix in https://github.com/bharatviswa504/hadoop-ozone/commit/b0d9df21ea9fec8365a2da45451db44d881213a2 . Thank you.




-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   OK, I agree with your idea of preserving versioning code. I'll add some work on this, but, AFAIK, such configuration to switch versioning on/off doesn't exist yet. We have to add such property in ozone-site.xml. I want to discuss the name of property before starting my work. It will be hidden by default, but it won't be good idea to change the name afterwards.
   
   Concurrently I'll take a look on OmFileCreateRequest and prepare update.
   
   Also, I have another question on `KeyManager` - it isn't used any more, but should it be updated and tested as well?


-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   Thank You @kuenishi for the contribution and diligently addressing reviewing comments.


-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   @kuenishi I was busy with some other tasks, will get to this today, 


-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -192,6 +195,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // If bucket versioning is turned on during the update, between key
+      // creation and key commit, old versions will be just overwritten and
+      // not kept. Bucket versioning will be effective from the first key
+      // creation after the knob turned on.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbOzoneKey,
+          omMetadataManager, omBucketInfo.getIsVersionEnabled(), trxnLogIndex,
+          ozoneManager.isRatisEnabled());
+      OmKeyInfo keyToDelete =

Review comment:
       I mean previously we donot read from keyTable with this change we are reading from keyTable. So it has some performance impact when compared to current code.
   
   But to solve this issue, we need 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.

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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -191,6 +195,21 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // TODO(HDDS-5461):  Currently, old versions are all moved to the delete
+      //  table. To preserve old version(s) in the key table on overwrite,
+      //  merge old versions to omKeyInfo here instead of moving the to the
+      //  delete table.
+      //  If bucket versioning is turned on somehow, overwritten blocks will
+      //  leak and never collected by block deletion service. Keep it off.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbOzoneKey,
+          omMetadataManager, omBucketInfo.getIsVersionEnabled(), trxnLogIndex,
+          ozoneManager.isRatisEnabled());
+      if (omMetadataManager.getKeyTable().isExist(dbOzoneKey)
+              && omBucketInfo.getIsVersionEnabled()) {
+        // TODO: delete this warning after versioning supported.
+        LOG.warn("Overwritten blocks of {}/{}/{} may have leaked.",

Review comment:
       I thought there was a potential race, but it wasn't as far as I reconsidered. I've removed this section.




-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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






-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
##########
@@ -442,12 +442,20 @@ public static File createOMDir(String dirPath) {
    *                     of the MultipartUploadAbort request which needs to
    *                     be set as the updateID of the partKeyInfos.
    *                     For regular Key deletes, this value should be set to
-   *                     the same updaeID as is in keyInfo.
+   *                     the same updateID as is in keyInfo.
    * @return {@link RepeatedOmKeyInfo}
    */
   public static RepeatedOmKeyInfo prepareKeyForDelete(OmKeyInfo keyInfo,
       RepeatedOmKeyInfo repeatedOmKeyInfo, long trxnLogIndex,
       boolean isRatisEnabled) {
+
+    // Set the updateID
+    keyInfo.setUpdateID(trxnLogIndex, isRatisEnabled);
+    return prepareKeyForDeleteWithoutUpdateID(keyInfo, repeatedOmKeyInfo);

Review comment:
       Not understood the reasoning for split of this method?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -156,6 +157,9 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               volumeName, bucketName);
 
       validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+      omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
+      // validateBucketAndVolume guarantees bucket existence and it's not null
+      assert omBucketInfo != null;

Review comment:
       Minor NIT: We donot need this. As validateBucketAndVolume throws error when bucket does not exist.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
##########
@@ -286,8 +287,17 @@ public synchronized long addNewVersion(
       // it is important that the new version are always at the tail of the list
       OmKeyLocationInfoGroup currentLatestVersion =
           keyLocationVersions.get(keyLocationVersions.size() - 1);
+
+      // The new version is created based on the latest version number
+      // and will not include key locations of old versions, until object

Review comment:
       This comment is confusing.
   
   The latest version holding previous block versions is removed as part 
   HDDS-5472. Old versions of location in OmKeyLocationInfoGroup causes OOM of OM (#2448)
   
   And now also when keepOldVersions is false, we create a new version. (With only that version blocks)

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponseWithFSO.java
##########
@@ -32,19 +33,21 @@
 
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
 
 /**
  * Response for CommitKey request - prefix layout1.
  */
-@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, FILE_TABLE})
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, FILE_TABLE, DELETED_TABLE})
 public class OMKeyCommitResponseWithFSO extends OMKeyCommitResponse {
 
   public OMKeyCommitResponseWithFSO(@Nonnull OMResponse omResponse,
-                               @Nonnull OmKeyInfo omKeyInfo,
-                               String ozoneKeyName, String openKeyName,
-                               @Nonnull OmBucketInfo omBucketInfo) {
+                                    @Nonnull OmKeyInfo omKeyInfo,

Review comment:
       Indentation is incorrect

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -191,6 +195,21 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // TODO(HDDS-5461):  Currently, old versions are all moved to the delete
+      //  table. To preserve old version(s) in the key table on overwrite,
+      //  merge old versions to omKeyInfo here instead of moving the to the
+      //  delete table.

Review comment:
       This comment is not clear to me.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
##########
@@ -143,6 +149,22 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // TODO(HDDS-5461):  Currently, old versions are all moved to the delete
+      //  table. To preserve old version(s) in the key table on overwrite,
+      //  merge old versions to omKeyInfo here instead of moving the to the
+      //  delete table.
+      //  If bucket versioning is turned on somehow, overwritten blocks will
+      //  leak and never collected by block deletion service. Keep it off.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbFileKey,
+              omMetadataManager, omBucketInfo.getIsVersionEnabled(),
+              trxnLogIndex, ozoneManager.isRatisEnabled());
+      if (omMetadataManager.getKeyTable().isExist(dbFileKey)
+              && omBucketInfo.getIsVersionEnabled()) {
+        // TODO: delete this warning after versioning supported.
+        LOG.warn("Overwritten blocks of {}/{}/{} may have leaked.",
+                volumeName, bucketName, keyName);
+      }
+
       // Add to cache of open key table and key table.
       OMFileRequest.addOpenFileTableCacheEntry(omMetadataManager, dbFileKey,

Review comment:
       ```
         if (keysToDelete != null) {
           omMetadataManager.getDeletedTable().addCacheEntry(
               new CacheKey<>(dbOzoneKey),
               new CacheValue<>(Optional.of(keysToDelete), trxnLogIndex));
         }
   ```
   
   Something like this

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -239,6 +259,40 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
     return omClientResponse;
   }
 
+  /**
+   * Prepare key for deletion service on overwrite.
+   *
+   * @param dbOzoneKey key to point to an object in RocksDB
+   * @param omMetadataManager
+   * @param isVersionEnabled
+   * @param trxnLogIndex
+   * @param isRatisEnabled
+   * @return Old keys eligible for deletion.
+   * @throws IOException
+   */
+  protected RepeatedOmKeyInfo getOldVersionsToCleanUp(
+          String dbOzoneKey, OMMetadataManager omMetadataManager,
+          boolean isVersionEnabled, long trxnLogIndex,
+          boolean isRatisEnabled) throws IOException {
+    if (isVersionEnabled) {
+      // Nothing to clean up in case versioning is on.
+      return null;
+    }
+    // Past keys that was deleted but still in deleted table,
+    // waiting for deletion service.
+    RepeatedOmKeyInfo keysToDelete =
+            omMetadataManager.getDeletedTable().getReadCopy(dbOzoneKey);

Review comment:
       Here we can use normal get(), as delete entries are not added to cache at all.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
##########
@@ -286,8 +287,17 @@ public synchronized long addNewVersion(
       // it is important that the new version are always at the tail of the list
       OmKeyLocationInfoGroup currentLatestVersion =
           keyLocationVersions.get(keyLocationVersions.size() - 1);
+
+      // The new version is created based on the latest version number
+      // and will not include key locations of old versions, until object
+      // versioning is supported and enabled.
       OmKeyLocationInfoGroup newVersion =
           currentLatestVersion.generateNextVersion(newLocationList);
+      if (!keepOldVersions) {
+        // Even though old versions are cleared here, they will be
+        // moved to delete table at the time of key commit
+        keyLocationVersions.clear();
+      }
       keyLocationVersions.add(newVersion);

Review comment:
       And also when oldVersions is false, when a key is override, the version of that should be zero right?
   (Now with this it is still last version + 1) 

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java
##########
@@ -29,28 +30,31 @@
 import java.io.IOException;
 import javax.annotation.Nonnull;
 
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_KEY_TABLE;
 
 /**
  * Response for CommitKey request.
  */
-@CleanupTableInfo(cleanupTables = {OPEN_KEY_TABLE, KEY_TABLE})
+@CleanupTableInfo(cleanupTables = {OPEN_KEY_TABLE, KEY_TABLE, DELETED_TABLE})

Review comment:
       Do we need to add DELETED_TABLE here?
   As we donot add any add any entries to delete table cache in OMKeyCommitRequest.java
   

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
##########
@@ -619,11 +619,11 @@ public void commitKey(OmKeyArgs args, long clientID) throws IOException {
       //update the block length for each block
       keyInfo.updateLocationInfoList(locationInfoList, false);
       metadataManager.getStore().move(
-          openKey,
-          objectKey,
-          keyInfo,
-          metadataManager.getOpenKeyTable(),
-          metadataManager.getKeyTable());
+              openKey,

Review comment:
       NIT: Spacing is wrong here

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
##########
@@ -143,6 +149,22 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // TODO(HDDS-5461):  Currently, old versions are all moved to the delete
+      //  table. To preserve old version(s) in the key table on overwrite,
+      //  merge old versions to omKeyInfo here instead of moving the to the
+      //  delete table.
+      //  If bucket versioning is turned on somehow, overwritten blocks will
+      //  leak and never collected by block deletion service. Keep it off.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbFileKey,
+              omMetadataManager, omBucketInfo.getIsVersionEnabled(),
+              trxnLogIndex, ozoneManager.isRatisEnabled());
+      if (omMetadataManager.getKeyTable().isExist(dbFileKey)
+              && omBucketInfo.getIsVersionEnabled()) {
+        // TODO: delete this warning after versioning supported.

Review comment:
       Same as above:
   Not sure why this blocks will be leaked, if we move the entire OmKeyInfo to delete table during delete.
   Deletion logic should consider all versions in OmKeyInfo and send those blocks. (If this is missing we should open a bug for this)

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java
##########
@@ -143,16 +143,16 @@ public static OmKeyLocationInfoGroup getFromProtobuf(
   }
 
   /**
-   * Given a new block location, generate a new version list based upon this
-   * one.
+   * Given a new block location, generate a new version list based upon the
+   * version number of this object. It does not keep past blocks.
    *
    * @param newLocationList a list of new location to be added.
    * @return newly generated OmKeyLocationInfoGroup
    */
   OmKeyLocationInfoGroup generateNextVersion(
       List<OmKeyLocationInfo> newLocationList) {
-    Map<Long, List<OmKeyLocationInfo>> newMap =
-        new HashMap<>();
+    Map<Long, List<OmKeyLocationInfo>> newMap = new HashMap<>();

Review comment:
       Minor: Can we avoid this unnecessary change.
   So, that it will be easy to track history.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java
##########
@@ -621,13 +621,20 @@ protected OmKeyInfo prepareFileInfo(
       //TODO args.getMetadata
     }
     if (dbKeyInfo != null) {
-      // TODO: Need to be fixed, as when key already exists, we are
-      //  appending new blocks to existing key.
-      // The key already exist, the new blocks will be added as new version
-      // when locations.size = 0, the new version will have identical blocks
-      // as its previous version
-      dbKeyInfo.addNewVersion(locations, false);
-      dbKeyInfo.setDataSize(size + dbKeyInfo.getDataSize());
+      // The key already exist, the new blocks will replace old ones
+      // as new versions unless the bucket does not have versioning
+      // turned on. If the versioning is on, the new version will have

Review comment:
       If the versioning is on, the new version will have
         // identical blocks as its previous version.
   This is not correct, this has been fixed already. Now each version has its own blocks.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -191,6 +195,21 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // TODO(HDDS-5461):  Currently, old versions are all moved to the delete
+      //  table. To preserve old version(s) in the key table on overwrite,
+      //  merge old versions to omKeyInfo here instead of moving the to the
+      //  delete table.
+      //  If bucket versioning is turned on somehow, overwritten blocks will
+      //  leak and never collected by block deletion service. Keep it off.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbOzoneKey,
+          omMetadataManager, omBucketInfo.getIsVersionEnabled(), trxnLogIndex,
+          ozoneManager.isRatisEnabled());
+      if (omMetadataManager.getKeyTable().isExist(dbOzoneKey)
+              && omBucketInfo.getIsVersionEnabled()) {
+        // TODO: delete this warning after versioning supported.
+        LOG.warn("Overwritten blocks of {}/{}/{} may have leaked.",

Review comment:
       Not sure why this blocks will be leaked, if we move the entire OmKeyInfo to delete table.
   Deletion logic should consider all versions in OmKeyInfo and send those blocks. (If this is missing we should open a bug for this)

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java
##########
@@ -94,4 +100,12 @@ protected OmBucketInfo getOmBucketInfo() {
   protected String getOzoneKeyName() {
     return ozoneKeyName;
   }
+
+  protected void maybeUpdateDeletedTable(OMMetadataManager omMetadataManager,

Review comment:
       Minor: Can we rename this as updateDeletedTable?

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java
##########
@@ -275,6 +276,76 @@ public void testValidateAndUpdateCacheWithKeyNotFound() throws Exception {
     Assert.assertNull(omKeyInfo);
   }
 
+  @Test
+  public void testValidateAndUpdateCacheOnOverwrite() throws Exception {
+    testValidateAndUpdateCache();
+
+    // Become a new client and set next version number
+    clientID = Time.now();
+    version += 1;
+
+    OMRequest modifiedOmRequest = doPreExecute(createCommitKeyRequest());
+
+    OMKeyCommitRequest omKeyCommitRequest =
+            getOmKeyCommitRequest(modifiedOmRequest);
+
+    KeyArgs keyArgs = modifiedOmRequest.getCommitKeyRequest().getKeyArgs();
+
+    // Key should be there in key table, as validateAndUpdateCache is called.
+    OmKeyInfo omKeyInfo =
+        omMetadataManager.getKeyTable().get(getOzonePathKey());
+
+    Assert.assertNotNull(omKeyInfo);
+    // Previously committed version
+    Assert.assertEquals(0L,
+            omKeyInfo.getLatestVersionLocations().getVersion());
+
+    // Append new blocks
+    List<OmKeyLocationInfo> allocatedLocationList =
+            keyArgs.getKeyLocationsList().stream()
+                    .map(OmKeyLocationInfo::getFromProtobuf)
+                    .collect(Collectors.toList());
+    addKeyToOpenKeyTable(allocatedLocationList);
+
+    OMClientResponse omClientResponse =
+            omKeyCommitRequest.validateAndUpdateCache(ozoneManager,
+                    102L, ozoneManagerDoubleBufferHelper);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+            omClientResponse.getOMResponse().getStatus());
+
+    String ozoneKey = getOzonePathKey();
+    // Entry should be deleted from openKey Table.

Review comment:
       This entry will not exist at all in openKeyTable right?
   As in KeyTable entry is with sessionid appended. And here we are checking ozone key.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -239,6 +259,40 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
     return omClientResponse;
   }
 
+  /**
+   * Prepare key for deletion service on overwrite.
+   *
+   * @param dbOzoneKey key to point to an object in RocksDB
+   * @param omMetadataManager
+   * @param isVersionEnabled
+   * @param trxnLogIndex
+   * @param isRatisEnabled
+   * @return Old keys eligible for deletion.
+   * @throws IOException
+   */
+  protected RepeatedOmKeyInfo getOldVersionsToCleanUp(
+          String dbOzoneKey, OMMetadataManager omMetadataManager,
+          boolean isVersionEnabled, long trxnLogIndex,
+          boolean isRatisEnabled) throws IOException {
+    if (isVersionEnabled) {
+      // Nothing to clean up in case versioning is on.
+      return null;
+    }
+    // Past keys that was deleted but still in deleted table,
+    // waiting for deletion service.
+    RepeatedOmKeyInfo keysToDelete =
+            omMetadataManager.getDeletedTable().getReadCopy(dbOzoneKey);
+    // Current key to be overwritten
+    OmKeyInfo keyToDelete =
+            omMetadataManager.getKeyTable().getReadCopy(dbOzoneKey);

Review comment:
       We should use get(). Because if we use getReadCopy() if it is in cache it might cause issues. (As if any changes happen to that object later)

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponseWithFSO.java
##########
@@ -32,19 +33,21 @@
 
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
 
 /**
  * Response for CommitKey request - prefix layout1.
  */
-@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, FILE_TABLE})
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, FILE_TABLE, DELETED_TABLE})

Review comment:
       Same as above for DELETED_TABLE

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
##########
@@ -143,6 +149,22 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // TODO(HDDS-5461):  Currently, old versions are all moved to the delete
+      //  table. To preserve old version(s) in the key table on overwrite,
+      //  merge old versions to omKeyInfo here instead of moving the to the
+      //  delete table.
+      //  If bucket versioning is turned on somehow, overwritten blocks will
+      //  leak and never collected by block deletion service. Keep it off.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbFileKey,
+              omMetadataManager, omBucketInfo.getIsVersionEnabled(),
+              trxnLogIndex, ozoneManager.isRatisEnabled());
+      if (omMetadataManager.getKeyTable().isExist(dbFileKey)
+              && omBucketInfo.getIsVersionEnabled()) {
+        // TODO: delete this warning after versioning supported.
+        LOG.warn("Overwritten blocks of {}/{}/{} may have leaked.",
+                volumeName, bucketName, keyName);
+      }
+
       // Add to cache of open key table and key table.
       OMFileRequest.addOpenFileTableCacheEntry(omMetadataManager, dbFileKey,

Review comment:
       I think how we add keyInfo to cache similarly we need to add delete entries to delete table cache.
   
   Lets say key created/committed we added to cache not yet committed to DB
   Before commit, if key is override even if we call get() we donot get anything as the entry is not in cache and double buffer flush not yet committed to DB. So, i think we need to add delete entry also cache.
   
   And leave the entry of DELETED_TABLE in cleanup in Response classes.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
##########
@@ -619,11 +619,11 @@ public void commitKey(OmKeyArgs args, long clientID) throws IOException {
       //update the block length for each block
       keyInfo.updateLocationInfoList(locationInfoList, false);
       metadataManager.getStore().move(
-          openKey,
-          objectKey,
-          keyInfo,
-          metadataManager.getOpenKeyTable(),
-          metadataManager.getKeyTable());
+              openKey,

Review comment:
       And also this is unncessary change, revert it back

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -191,6 +195,21 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // TODO(HDDS-5461):  Currently, old versions are all moved to the delete
+      //  table. To preserve old version(s) in the key table on overwrite,
+      //  merge old versions to omKeyInfo here instead of moving the to the
+      //  delete table.

Review comment:
       In simple we can mention
   If versioning is enabled, if key is override we move the old copy of object to delete table. (Something like that)




-- 
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] bharatviswa504 edited a comment on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   > OK, I agree with your idea of preserving versioning code. I'll add some work on this, but, AFAIK, such configuration to switch versioning on/off doesn't exist yet. We have to add such property in ozone-site.xml. I want to discuss the name of property before starting my work. It will be hidden by default, but it won't be good idea to change the name afterwards.
   > 
   > Concurrently I'll take a look on OmFileCreateRequest and prepare update.
   > 
   > Also, I have another question on `KeyManager` - it isn't used any more, but should it be updated and tested as well?
   
   We have isVersionEnabled flag already, no where we use it on the server. Do you think still we need to have config for this?
   ```
   message BucketInfo {
       required string volumeName = 1;
       required string bucketName = 2;
       repeated OzoneAclInfo acls = 3;
       required bool isVersionEnabled = 4 [default = false];
   ```


-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
##########
@@ -442,12 +442,20 @@ public static File createOMDir(String dirPath) {
    *                     of the MultipartUploadAbort request which needs to
    *                     be set as the updateID of the partKeyInfos.
    *                     For regular Key deletes, this value should be set to
-   *                     the same updaeID as is in keyInfo.
+   *                     the same updateID as is in keyInfo.
    * @return {@link RepeatedOmKeyInfo}
    */
   public static RepeatedOmKeyInfo prepareKeyForDelete(OmKeyInfo keyInfo,
       RepeatedOmKeyInfo repeatedOmKeyInfo, long trxnLogIndex,
       boolean isRatisEnabled) {
+
+    // Set the updateID
+    keyInfo.setUpdateID(trxnLogIndex, isRatisEnabled);
+    return prepareKeyForDeleteWithoutUpdateID(keyInfo, repeatedOmKeyInfo);

Review comment:
       Sorry it was the remain of my trial-and-error through working on this task. I'll remove this split.




-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   > Note: @bharatviswa504 has been also working on this concurrently in #2432 .
   
   As mentioned in the Jira, my PR addresses during key create return only latest version of key. 
   
   Key1 create - Blocks Version0 B1(0), B2(0), B3(0)
   
   1st time override
   Key1 override - Version0 -> B1(0), B2(0), B3(0), Version1 -> B1(0), B2(0), B3(0), New Blocks for this version (B4(1), B5(1), B6(1))
   
   2nd time override
   Key1 override - Version0 B1(0), B2(0), B3(0), Version1 -> B1(0), B2(0), B3(0), New Blocks for this version (B4(1), B5(1), B6(1)), Version 2 -> B1(0), B2(0), B3(0), B4(1), B5(1), B6(1), B7(2), B8(2), B9(2)
   
   So, the problem is each time a key is override, each version has its version of blocks along with all older versions.
   
   So, now when key create it returns all this versions. So, the list can become huge in size, depending on number of times key is override. In this patch I will fix to return latest version for create key. 
   
   >(I will open a new Jira to discuss about having each version including all its previous version)
   
   This PR is proposing to solve the 2nd part of the problem.


-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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






-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -156,6 +157,9 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               volumeName, bucketName);
 
       validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+      omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
+      // validateBucketAndVolume guarantees bucket existence and it's not null
+      assert omBucketInfo != null;

Review comment:
       Good catch. Thank you.




-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
##########
@@ -143,6 +149,22 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // TODO(HDDS-5461):  Currently, old versions are all moved to the delete
+      //  table. To preserve old version(s) in the key table on overwrite,
+      //  merge old versions to omKeyInfo here instead of moving the to the
+      //  delete table.
+      //  If bucket versioning is turned on somehow, overwritten blocks will
+      //  leak and never collected by block deletion service. Keep it off.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbFileKey,
+              omMetadataManager, omBucketInfo.getIsVersionEnabled(),
+              trxnLogIndex, ozoneManager.isRatisEnabled());
+      if (omMetadataManager.getKeyTable().isExist(dbFileKey)
+              && omBucketInfo.getIsVersionEnabled()) {
+        // TODO: delete this warning after versioning supported.
+        LOG.warn("Overwritten blocks of {}/{}/{} may have leaked.",
+                volumeName, bucketName, keyName);
+      }
+
       // Add to cache of open key table and key table.
       OMFileRequest.addOpenFileTableCacheEntry(omMetadataManager, dbFileKey,

Review comment:
       Thank you for clarifying the relation between cleanup in Response classes and table caches. I think fixed all of them and now they should be consistent.




-- 
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] vivekratnavel commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   @bharatviswa504 @ChenSammi Can you please take another look at this patch and help merge 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.

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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   Thank you for kind and sole review, again and again. I think I've addressed all comments.


-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -239,6 +259,40 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
     return omClientResponse;
   }
 
+  /**
+   * Prepare key for deletion service on overwrite.
+   *
+   * @param dbOzoneKey key to point to an object in RocksDB
+   * @param omMetadataManager
+   * @param isVersionEnabled
+   * @param trxnLogIndex
+   * @param isRatisEnabled
+   * @return Old keys eligible for deletion.
+   * @throws IOException
+   */
+  protected RepeatedOmKeyInfo getOldVersionsToCleanUp(
+          String dbOzoneKey, OMMetadataManager omMetadataManager,
+          boolean isVersionEnabled, long trxnLogIndex,
+          boolean isRatisEnabled) throws IOException {
+    if (isVersionEnabled) {
+      // Nothing to clean up in case versioning is on.
+      return null;
+    }
+    // Past keys that was deleted but still in deleted table,
+    // waiting for deletion service.
+    RepeatedOmKeyInfo keysToDelete =
+            omMetadataManager.getDeletedTable().getReadCopy(dbOzoneKey);

Review comment:
       No we should use normal get. ReadCopy should be used only when we donot plan to change the returned value.
   Cases like getBucket, getVol where we check returned value from cache just to check existence, but here that is not the case




-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -192,6 +195,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // If bucket versioning is turned on during the update, between key
+      // creation and key commit, old versions will be just overwritten and
+      // not kept. Bucket versioning will be effective from the first key
+      // creation after the knob turned on.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbOzoneKey,
+          omMetadataManager, omBucketInfo.getIsVersionEnabled(), trxnLogIndex,
+          ozoneManager.isRatisEnabled());
+      OmKeyInfo keyToDelete =

Review comment:
       I mean previously we donot read from keyTable with this change we are reading from keyTable. So it has some performance impact when compared to current code.
   
   But to solve this issue, we need it.
   
   My comment is more of to say, to address this issue we need one extra DB read




-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   > And also it needs similar update for File requests also (OMFileCreateRequest)
   
   As the code update is all in  `OMKeyRequest#prepareFileInfo()`, OMFileCreateRequest shares code of preparing OmKeyInfo with OMKeyCreateRequest and OMkeyCreateRequestWithFSO. 


-- 
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] kuenishi edited a comment on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   ~~The checkstyle is failing in test code that is not related to this patch:~~
   ```
   hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
    3681: Line is longer than 80 characters (found 95).
    3703: Line is longer than 80 characters (found 82).
    3706: Line is longer than 80 characters (found 90).
    3711: Line is longer than 80 characters (found 86).
    3715: Line is longer than 80 characters (found 90).
   ```


-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   The checkstyle is failing in test code that is not related to this patch:
   ```
   hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
    3681: Line is longer than 80 characters (found 95).
    3703: Line is longer than 80 characters (found 82).
    3706: Line is longer than 80 characters (found 90).
    3711: Line is longer than 80 characters (found 86).
    3715: Line is longer than 80 characters (found 90).
   ```


-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -192,6 +195,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // If bucket versioning is turned on during the update, between key
+      // creation and key commit, old versions will be just overwritten and
+      // not kept. Bucket versioning will be effective from the first key
+      // creation after the knob turned on.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbOzoneKey,
+          omMetadataManager, omBucketInfo.getIsVersionEnabled(), trxnLogIndex,
+          ozoneManager.isRatisEnabled());
+      OmKeyInfo keyToDelete =

Review comment:
       With this we need extra DB op for read. I see with this change it is required,




-- 
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] bharatviswa504 edited a comment on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   I have left the comments open which needs to be addressed to make it easy, remaining all I have resolved


-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java
##########
@@ -143,16 +143,16 @@ public static OmKeyLocationInfoGroup getFromProtobuf(
   }
 
   /**
-   * Given a new block location, generate a new version list based upon this
-   * one.
+   * Given a new block location, generate a new version list based upon the
+   * version number of this object. It does not keep past blocks.
    *
    * @param newLocationList a list of new location to be added.
    * @return newly generated OmKeyLocationInfoGroup
    */
   OmKeyLocationInfoGroup generateNextVersion(
       List<OmKeyLocationInfo> newLocationList) {
-    Map<Long, List<OmKeyLocationInfo>> newMap =
-        new HashMap<>();
+    Map<Long, List<OmKeyLocationInfo>> newMap = new HashMap<>();

Review comment:
       Sorry, I cleared the clutter.




-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   Kicked off another CI run, will merge this after clean CI.
   Thank You @kuenishi for the nice improvement which is very much needed in OM.


-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
##########
@@ -286,8 +287,17 @@ public synchronized long addNewVersion(
       // it is important that the new version are always at the tail of the list
       OmKeyLocationInfoGroup currentLatestVersion =
           keyLocationVersions.get(keyLocationVersions.size() - 1);
+
+      // The new version is created based on the latest version number
+      // and will not include key locations of old versions, until object
+      // versioning is supported and enabled.
       OmKeyLocationInfoGroup newVersion =
           currentLatestVersion.generateNextVersion(newLocationList);
+      if (!keepOldVersions) {
+        // Even though old versions are cleared here, they will be
+        // moved to delete table at the time of key commit
+        keyLocationVersions.clear();
+      }
       keyLocationVersions.add(newVersion);

Review comment:
       So even though versioning is disabled, we have a key with version 2 if key is override 2 times?
   I am okay with it, as when versioning is supported we can revisit.




-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java
##########
@@ -275,6 +276,76 @@ public void testValidateAndUpdateCacheWithKeyNotFound() throws Exception {
     Assert.assertNull(omKeyInfo);
   }
 
+  @Test
+  public void testValidateAndUpdateCacheOnOverwrite() throws Exception {
+    testValidateAndUpdateCache();
+
+    // Become a new client and set next version number
+    clientID = Time.now();
+    version += 1;
+
+    OMRequest modifiedOmRequest = doPreExecute(createCommitKeyRequest());
+
+    OMKeyCommitRequest omKeyCommitRequest =
+            getOmKeyCommitRequest(modifiedOmRequest);
+
+    KeyArgs keyArgs = modifiedOmRequest.getCommitKeyRequest().getKeyArgs();
+
+    // Key should be there in key table, as validateAndUpdateCache is called.
+    OmKeyInfo omKeyInfo =
+        omMetadataManager.getKeyTable().get(getOzonePathKey());
+
+    Assert.assertNotNull(omKeyInfo);
+    // Previously committed version
+    Assert.assertEquals(0L,
+            omKeyInfo.getLatestVersionLocations().getVersion());
+
+    // Append new blocks
+    List<OmKeyLocationInfo> allocatedLocationList =
+            keyArgs.getKeyLocationsList().stream()
+                    .map(OmKeyLocationInfo::getFromProtobuf)
+                    .collect(Collectors.toList());
+    addKeyToOpenKeyTable(allocatedLocationList);
+
+    OMClientResponse omClientResponse =
+            omKeyCommitRequest.validateAndUpdateCache(ozoneManager,
+                    102L, ozoneManagerDoubleBufferHelper);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+            omClientResponse.getOMResponse().getStatus());
+
+    String ozoneKey = getOzonePathKey();
+    // Entry should be deleted from openKey Table.

Review comment:
       This is still not yet addressed? See above comment for reference




-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java
##########
@@ -275,6 +276,76 @@ public void testValidateAndUpdateCacheWithKeyNotFound() throws Exception {
     Assert.assertNull(omKeyInfo);
   }
 
+  @Test
+  public void testValidateAndUpdateCacheOnOverwrite() throws Exception {
+    testValidateAndUpdateCache();
+
+    // Become a new client and set next version number
+    clientID = Time.now();
+    version += 1;
+
+    OMRequest modifiedOmRequest = doPreExecute(createCommitKeyRequest());
+
+    OMKeyCommitRequest omKeyCommitRequest =
+            getOmKeyCommitRequest(modifiedOmRequest);
+
+    KeyArgs keyArgs = modifiedOmRequest.getCommitKeyRequest().getKeyArgs();
+
+    // Key should be there in key table, as validateAndUpdateCache is called.
+    OmKeyInfo omKeyInfo =
+        omMetadataManager.getKeyTable().get(getOzonePathKey());
+
+    Assert.assertNotNull(omKeyInfo);
+    // Previously committed version
+    Assert.assertEquals(0L,
+            omKeyInfo.getLatestVersionLocations().getVersion());
+
+    // Append new blocks
+    List<OmKeyLocationInfo> allocatedLocationList =
+            keyArgs.getKeyLocationsList().stream()
+                    .map(OmKeyLocationInfo::getFromProtobuf)
+                    .collect(Collectors.toList());
+    addKeyToOpenKeyTable(allocatedLocationList);
+
+    OMClientResponse omClientResponse =
+            omKeyCommitRequest.validateAndUpdateCache(ozoneManager,
+                    102L, ozoneManagerDoubleBufferHelper);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+            omClientResponse.getOMResponse().getStatus());
+
+    String ozoneKey = getOzonePathKey();
+    // Entry should be deleted from openKey Table.

Review comment:
       This line was actually copy & pasted from ` testValidateAndUpdateCache()` (line 152-153), but as this is not the test for overwriting, I'm removing 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.

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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   @kuenishi 
   Have you got a chance to look at latest comments. Let me know if any help is needed?


-- 
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] bharatviswa504 edited a comment on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   > OK, I agree with your idea of preserving versioning code. I'll add some work on this, but, AFAIK, such configuration to switch versioning on/off doesn't exist yet. We have to add such property in ozone-site.xml. I want to discuss the name of property before starting my work. It will be hidden by default, but it won't be good idea to change the name afterwards.
   > 
   > Concurrently I'll take a look on OmFileCreateRequest and prepare update.
   > 
   > Also, I have another question on `KeyManager` - it isn't used any more, but should it be updated and tested as well?
   
   We have isVersionEnabled flag already, no where we use it on the server. Do you think still we need to have config for this?
   ```
   message BucketInfo {
       required string volumeName = 1;
       required string bucketName = 2;
       repeated OzoneAclInfo acls = 3;
       required bool isVersionEnabled = 4 [default = false];
   ```


-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   @bharatviswa504 Thank you for the review. I think I addressed all of your comment. 
   
   I thought there was a race condition around switching bucket versioning on and off, with keys created and committed, but I was wrong and there would be no leak of blocks as far as I understand.


-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   I think I've addressed all comments, but I'll need a bit more hustle to pass the tests.


-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   @bharatviswa504 So now it's all green and I addressed your reviews. Please check it out, again!


-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   > Do you think still we need to have config for this?
   
   No, we don't. Thank you for the guidance.


-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -239,6 +259,40 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
     return omClientResponse;
   }
 
+  /**
+   * Prepare key for deletion service on overwrite.
+   *
+   * @param dbOzoneKey key to point to an object in RocksDB
+   * @param omMetadataManager
+   * @param isVersionEnabled
+   * @param trxnLogIndex
+   * @param isRatisEnabled
+   * @return Old keys eligible for deletion.
+   * @throws IOException
+   */
+  protected RepeatedOmKeyInfo getOldVersionsToCleanUp(
+          String dbOzoneKey, OMMetadataManager omMetadataManager,
+          boolean isVersionEnabled, long trxnLogIndex,
+          boolean isRatisEnabled) throws IOException {
+    if (isVersionEnabled) {
+      // Nothing to clean up in case versioning is on.
+      return null;
+    }
+    // Past keys that was deleted but still in deleted table,
+    // waiting for deletion service.
+    RepeatedOmKeyInfo keysToDelete =
+            omMetadataManager.getDeletedTable().getReadCopy(dbOzoneKey);

Review comment:
       No we should use normal get. ReadCopy should be used only when we donot plan to change.
   Cases like getBucket, getVol where we check returned value from cache just to check existence, but here that is not the case




-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponseWithFSO.java
##########
@@ -32,19 +33,21 @@
 
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
 import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE;
 
 /**
  * Response for CommitKey request - prefix layout1.
  */
-@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, FILE_TABLE})
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE, FILE_TABLE, DELETED_TABLE})
 public class OMKeyCommitResponseWithFSO extends OMKeyCommitResponse {
 
   public OMKeyCommitResponseWithFSO(@Nonnull OMResponse omResponse,
-                               @Nonnull OmKeyInfo omKeyInfo,
-                               String ozoneKeyName, String openKeyName,
-                               @Nonnull OmBucketInfo omBucketInfo) {
+                                    @Nonnull OmKeyInfo omKeyInfo,

Review comment:
       Still incorrect




-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   > OK, I agree with your idea of preserving versioning code. I'll add some work on this, but, AFAIK, such configuration to switch versioning on/off doesn't exist yet. We have to add such property in ozone-site.xml. I want to discuss the name of property before starting my work. It will be hidden by default, but it won't be good idea to change the name afterwards.
   > 
   > Concurrently I'll take a look on OmFileCreateRequest and prepare update.
   > 
   > Also, I have another question on `KeyManager` - it isn't used any more, but should it be updated and tested as well?
   
   We have isVersionEnabled flag already, no where we use it on the server. Do you think still we need to have config for this?
   message BucketInfo {
       required string volumeName = 1;
       required string bucketName = 2;
       repeated OzoneAclInfo acls = 3;
       required bool isVersionEnabled = 4 [default = false];


-- 
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] bharatviswa504 edited a comment on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   > OK, I agree with your idea of preserving versioning code. I'll add some work on this, but, AFAIK, such configuration to switch versioning on/off doesn't exist yet. We have to add such property in ozone-site.xml. I want to discuss the name of property before starting my work. It will be hidden by default, but it won't be good idea to change the name afterwards.
   > 
   > Concurrently I'll take a look on OmFileCreateRequest and prepare update.
   > 
   > Also, I have another question on `KeyManager` - it isn't used any more, but should it be updated and tested as well?
   
   We have isVersionEnabled flag already, no where we use it on the server. Do you think still we need to have config for this?
   ```
   message BucketInfo {
       required string volumeName = 1;
       required string bucketName = 2;
       repeated OzoneAclInfo acls = 3;
       required bool isVersionEnabled = 4 [default = false];
   ```


-- 
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] ChenSammi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java
##########
@@ -143,16 +143,22 @@ public static OmKeyLocationInfoGroup getFromProtobuf(
   }
 
   /**
-   * Given a new block location, generate a new version list based upon this
-   * one.
+   * Given a new block location, generate a new version list based upon the
+   * version number of this object. Keep the blocks to when keepVersions is
+   * on for object versioning.
    *
    * @param newLocationList a list of new location to be added.
+   * @param keepVersions a knob to carry current locations to the next version.
    * @return newly generated OmKeyLocationInfoGroup
    */
   OmKeyLocationInfoGroup generateNextVersion(
-      List<OmKeyLocationInfo> newLocationList) {
-    Map<Long, List<OmKeyLocationInfo>> newMap =
-        new HashMap<>();
+      List<OmKeyLocationInfo> newLocationList, boolean keepVersions) {
+    Map<Long, List<OmKeyLocationInfo>> newMap = null;
+    if (keepVersions) {
+      newMap = new HashMap<>(locationVersionMap);

Review comment:
       We don‘t need to include the locationVersionMap to the new  OmKeyLocationInfoGroup even when version is true.  
   The current OmKeyLocationInfoGroup design actually has one point to improve.   You can refer to @symious document https://docs.google.com/document/d/1UYVETpSHyVppNPjHq8VuXj2t2Y0LYQNRYa6awSTCROA/edit#heading=h.h2k2ovrsrw33 for more detail information. 




-- 
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] kuenishi commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   Note: @bharatviswa504 has been also working on this concurrently in #2432 .


-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
##########
@@ -286,8 +287,17 @@ public synchronized long addNewVersion(
       // it is important that the new version are always at the tail of the list
       OmKeyLocationInfoGroup currentLatestVersion =
           keyLocationVersions.get(keyLocationVersions.size() - 1);
+
+      // The new version is created based on the latest version number
+      // and will not include key locations of old versions, until object
+      // versioning is supported and enabled.
       OmKeyLocationInfoGroup newVersion =
           currentLatestVersion.generateNextVersion(newLocationList);
+      if (!keepOldVersions) {
+        // Even though old versions are cleared here, they will be
+        // moved to delete table at the time of key commit
+        keyLocationVersions.clear();
+      }
       keyLocationVersions.add(newVersion);

Review comment:
       I'm neutral on the version number on overwrite, from the viewpoint of algorithm. But I have an instinct on operating complex distributed systems like Ozone, the version numbers had better be monotonically increasing, especially when we debug production issues.
   
   I would also want to keep the version number even after the key is deleted, but that'd be another change of design and IMO it'd be out of the scope of this work.




-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -192,6 +195,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // If bucket versioning is turned on during the update, between key
+      // creation and key commit, old versions will be just overwritten and
+      // not kept. Bucket versioning will be effective from the first key
+      // creation after the knob turned on.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbOzoneKey,
+          omMetadataManager, omBucketInfo.getIsVersionEnabled(), trxnLogIndex,
+          ozoneManager.isRatisEnabled());
+      OmKeyInfo keyToDelete =

Review comment:
       Sorry, I don't get your comment. My guess is that some read operation should be added to the batch in OMKeyCommitResponse?
   
   If so, to protect the data from race conditions, I don't think the read op in the batch is needed because `validateAndUpdateCache(...)` is executed in a serialized manner at OMs.




-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -192,6 +195,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       // Set the UpdateID to current transactionLogIndex
       omKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
+      // If bucket versioning is turned on during the update, between key
+      // creation and key commit, old versions will be just overwritten and
+      // not kept. Bucket versioning will be effective from the first key
+      // creation after the knob turned on.
+      RepeatedOmKeyInfo keysToDelete = getOldVersionsToCleanUp(dbOzoneKey,
+          omMetadataManager, omBucketInfo.getIsVersionEnabled(), trxnLogIndex,
+          ozoneManager.isRatisEnabled());
+      OmKeyInfo keyToDelete =

Review comment:
       I got it, thank you. Maybe later, we can add some optimizations like Bloom filter to lower the read cost for keys that does not exist yet, if this read operation has major impact to write performance.




-- 
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] bharatviswa504 commented on pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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


   @kuenishi 
   Overall LGTM, few minor comments, once addressed good to go.


-- 
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] bharatviswa504 commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
##########
@@ -239,6 +259,40 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
     return omClientResponse;
   }
 
+  /**
+   * Prepare key for deletion service on overwrite.
+   *
+   * @param dbOzoneKey key to point to an object in RocksDB
+   * @param omMetadataManager
+   * @param isVersionEnabled
+   * @param trxnLogIndex
+   * @param isRatisEnabled
+   * @return Old keys eligible for deletion.
+   * @throws IOException
+   */
+  protected RepeatedOmKeyInfo getOldVersionsToCleanUp(
+          String dbOzoneKey, OMMetadataManager omMetadataManager,
+          boolean isVersionEnabled, long trxnLogIndex,
+          boolean isRatisEnabled) throws IOException {
+    if (isVersionEnabled) {
+      // Nothing to clean up in case versioning is on.
+      return null;
+    }
+    // Past keys that was deleted but still in deleted table,
+    // waiting for deletion service.
+    RepeatedOmKeyInfo keysToDelete =
+            omMetadataManager.getDeletedTable().getReadCopy(dbOzoneKey);
+    // Current key to be overwritten
+    OmKeyInfo keyToDelete =
+            omMetadataManager.getKeyTable().getReadCopy(dbOzoneKey);

Review comment:
       Need to be addressed yet




-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java
##########
@@ -143,16 +143,22 @@ public static OmKeyLocationInfoGroup getFromProtobuf(
   }
 
   /**
-   * Given a new block location, generate a new version list based upon this
-   * one.
+   * Given a new block location, generate a new version list based upon the
+   * version number of this object. Keep the blocks to when keepVersions is
+   * on for object versioning.
    *
    * @param newLocationList a list of new location to be added.
+   * @param keepVersions a knob to carry current locations to the next version.
    * @return newly generated OmKeyLocationInfoGroup
    */
   OmKeyLocationInfoGroup generateNextVersion(
-      List<OmKeyLocationInfo> newLocationList) {
-    Map<Long, List<OmKeyLocationInfo>> newMap =
-        new HashMap<>();
+      List<OmKeyLocationInfo> newLocationList, boolean keepVersions) {
+    Map<Long, List<OmKeyLocationInfo>> newMap = null;
+    if (keepVersions) {
+      newMap = new HashMap<>(locationVersionMap);

Review comment:
       Thank you for the review. I updated the commented  part so that it does keep past location version map and also added option 3 of my idea in this issue to the @symious document.
   
   I'm sorry that I don't have much time to add necessary unit tests and make integration tests passing for now, but will try in a few days.




-- 
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] kuenishi commented on a change in pull request #2433: HDDS-5461. Move old objects to delete table on overwrite

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java
##########
@@ -621,13 +621,20 @@ protected OmKeyInfo prepareFileInfo(
       //TODO args.getMetadata
     }
     if (dbKeyInfo != null) {
-      // TODO: Need to be fixed, as when key already exists, we are
-      //  appending new blocks to existing key.
-      // The key already exist, the new blocks will be added as new version
-      // when locations.size = 0, the new version will have identical blocks
-      // as its previous version
-      dbKeyInfo.addNewVersion(locations, false);
-      dbKeyInfo.setDataSize(size + dbKeyInfo.getDataSize());
+      // The key already exist, the new blocks will replace old ones
+      // as new versions unless the bucket does not have versioning
+      // turned on. If the versioning is on, the new version will have

Review comment:
       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