You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/04/26 02:01:13 UTC

[GitHub] [ozone] guihecheng opened a new pull request, #3346: HDDS-6542. KeyValueContainer operation adaptation for schema v3 containers.

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

   ## What changes were proposed in this pull request?
   
   KeyValueContainer operation adaptation for schema v3 containers, includes:
   - create & load
   - close
   - delete
   Not includes the following since they are in some dedicated standalone services:
   - block deletion
   - import & export
   
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6542
   
   ## How was this patch tested?
   
   Extending all container schema related UTs.
   


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

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 diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerMetadataInspector.java:
##########
@@ -44,9 +44,9 @@
     extends TestKeyValueContainerIntegrityChecks {
   private static final long CONTAINER_ID = 102;
 
-  public TestKeyValueContainerMetadataInspector(ContainerLayoutTestInfo
-      containerLayoutTestInfo) {
-    super(containerLayoutTestInfo);
+  public TestKeyValueContainerMetadataInspector(
+      ContainerTestVersionInfo versionInfo) {
+    super(versionInfo);

Review Comment:
   Why we don't create RocksDB for SchemaV3 in this test class? 



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

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

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


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


[GitHub] [ozone] ChenSammi merged pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


-- 
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 diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java:
##########
@@ -127,6 +131,7 @@ public void setUp() throws Exception {
     HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
         .getAbsolutePath()).conf(CONF).datanodeUuid(datanodeId
         .toString()).build();
+    StorageVolumeUtil.checkVolume(hddsVolume, scmId, scmId, CONF, null, null);

Review Comment:
   Use createDbInstancesForTestIfNeeded instead? 



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

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

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


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


[GitHub] [ozone] ChenSammi commented on pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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

   The patch LGTM, +1.  
   
   Thanks @guihecheng .


-- 
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] guihecheng commented on a diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerMetadataInspector.java:
##########
@@ -44,9 +44,9 @@
     extends TestKeyValueContainerIntegrityChecks {
   private static final long CONTAINER_ID = 102;
 
-  public TestKeyValueContainerMetadataInspector(ContainerLayoutTestInfo
-      containerLayoutTestInfo) {
-    super(containerLayoutTestInfo);
+  public TestKeyValueContainerMetadataInspector(
+      ContainerTestVersionInfo versionInfo) {
+    super(versionInfo);

Review Comment:
   Because this test class is a derived class from its base class `TestKeyValueContainerIntegrityChecks`, and the db instances are created there.



-- 
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] guihecheng commented on a diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java:
##########
@@ -271,6 +271,15 @@ List<? extends KeyValue<KEY, VALUE>> getSequentialRangeKVs(KEY startKey,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException;
 
+  /**
+   * Delete all keys with a prefix.
+   * @param batch
+   * @param prefix
+   * @return
+   */
+  void deleteAllWithPrefix(BatchOperation batch, KEY prefix)

Review Comment:
   deleteWithBatch seems to be an existing API, let's name it deleteBatchWithPrefix



-- 
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 diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java:
##########
@@ -237,4 +237,21 @@ public static void verifyBCSId(KeyValueContainer container, BlockID blockID)
               + containerBCSId + ".", UNKNOWN_BCSID);
     }
   }
+
+  /**
+   * Remove container KV metadata from per-disk db store.
+   * @param containerData
+   * @param conf
+   * @throws IOException
+   */
+  public static void dropAllFromDB(KeyValueContainerData containerData,

Review Comment:
   1.  dropAllFromDB -> removeContainerFromDB
   2.  removeDB(),  the logic of SCHEMA_V3 handling is not used. Better remove 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] ChenSammi commented on a diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java:
##########
@@ -942,9 +941,14 @@ public Container importContainer(ContainerData originalContainerData,
       final InputStream rawContainerStream,
       final TarContainerPacker packer)
       throws IOException {
+    Preconditions.checkState(originalContainerData instanceof
+        KeyValueContainerData, "Should be KeyValueContainerData instance");
 
     KeyValueContainerData containerData =
         new KeyValueContainerData(originalContainerData);
+    // schemaVersion should be used to decide the container db path
+    containerData.setSchemaVersion(
+        ((KeyValueContainerData) originalContainerData).getSchemaVersion());

Review Comment:
   Move this schemaVersion set into KeyValueContainerData(ContainerData data) constructor, change the constructor parameter type to KeyValueContainerData instead.



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

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

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


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


[GitHub] [ozone] guihecheng commented on a diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java:
##########
@@ -271,6 +271,15 @@ List<? extends KeyValue<KEY, VALUE>> getSequentialRangeKVs(KEY startKey,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException;
 
+  /**
+   * Delete all keys with a prefix.

Review Comment:
   Thanks, the new comment is better.



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java:
##########
@@ -271,6 +271,15 @@ List<? extends KeyValue<KEY, VALUE>> getSequentialRangeKVs(KEY startKey,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException;
 
+  /**
+   * Delete all keys with a prefix.
+   * @param batch
+   * @param prefix
+   * @return
+   */
+  void deleteAllWithPrefix(BatchOperation batch, KEY prefix)

Review Comment:
   OK.



-- 
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] guihecheng commented on pull request #3346: HDDS-6542. KeyValueContainer operation adaptation for schema v3 containers.

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

   @ChenSammi 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] guihecheng commented on a diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java:
##########
@@ -237,4 +237,21 @@ public static void verifyBCSId(KeyValueContainer container, BlockID blockID)
               + containerBCSId + ".", UNKNOWN_BCSID);
     }
   }
+
+  /**
+   * Remove container KV metadata from per-disk db store.
+   * @param containerData
+   * @param conf
+   * @throws IOException
+   */
+  public static void dropAllFromDB(KeyValueContainerData containerData,

Review Comment:
   Oh, I think for 2 I shall remove the logic indeed, because the disk failure case is handled in other places.



-- 
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 diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java:
##########
@@ -74,4 +75,15 @@ public BlockIterator<BlockData> getBlockIterator(long containerID,
         getBlockDataTableWithIterator()
             .iterator(getContainerKeyPrefix(containerID)), filter);
   }
+
+  public void dropAllWithPrefix(long containerID) throws IOException {

Review Comment:
   dropAllWithPrefix -> removeContainerFromDB



-- 
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 diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java:
##########
@@ -356,22 +379,25 @@ public void close() throws StorageContainerException {
       // Second sync should be a very light operation as sync has already
       // been done outside the lock.
       flushAndSyncDB();
-      updateContainerData(containerData::closeContainer);
+      updateContainerData(closer);
       clearPendingPutBlockCache();
     } finally {
       writeUnlock();
     }
-    LOG.info("Container {} is closed with bcsId {}.",
-        containerData.getContainerID(),
-        containerData.getBlockCommitSequenceId());
   }
 
-  @Override
-  public void updateDataScanTimestamp(Instant time)
+  /**
+   * For db-per-volume schemas, we don't flush and sync the whole db
+   * on closing of a single container.
+   * @param closer
+   * @throws StorageContainerException
+   */
+  private void closeWithoutFlush(Runnable closer)

Review Comment:
   Can we merge closeWithFlush and closeWithoutFlush into one function, using the parameter to distinguish flush or not flush? 



-- 
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 pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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

   @guihecheng , thanks for the working on this.  The overall patch looks good.  Just some comments about the function naming. 


-- 
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 diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java:
##########
@@ -73,23 +76,28 @@ public class TestBlockManagerImpl {
   private BlockID blockID1;
 
   private final ContainerLayoutVersion layout;
+  private final String schemaVersion;
 
-  public TestBlockManagerImpl(ContainerLayoutVersion layout) {
-    this.layout = layout;
+  public TestBlockManagerImpl(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    this.config = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, config);
   }
 
   @Parameterized.Parameters
   public static Iterable<Object[]> parameters() {
-    return ContainerLayoutTestInfo.containerLayoutParameters();
+    return ContainerTestVersionInfo.versionParameters();
   }
 
   @Before
   public void setUp() throws Exception {
-    config = new OzoneConfiguration();
     UUID datanodeId = UUID.randomUUID();
     HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
         .getAbsolutePath()).conf(config).datanodeUuid(datanodeId
         .toString()).build();
+    StorageVolumeUtil.checkVolume(hddsVolume, scmId, scmId, config,
+        null, null);

Review Comment:
   Shall we call createDbInstancesForTestIfNeeded instead?



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

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

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


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


[GitHub] [ozone] guihecheng commented on a diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java:
##########
@@ -942,9 +941,14 @@ public Container importContainer(ContainerData originalContainerData,
       final InputStream rawContainerStream,
       final TarContainerPacker packer)
       throws IOException {
+    Preconditions.checkState(originalContainerData instanceof
+        KeyValueContainerData, "Should be KeyValueContainerData instance");
 
     KeyValueContainerData containerData =
         new KeyValueContainerData(originalContainerData);
+    // schemaVersion should be used to decide the container db path
+    containerData.setSchemaVersion(
+        ((KeyValueContainerData) originalContainerData).getSchemaVersion());

Review Comment:
   Hmmm, I'll try to add schemaVersion to the constructor.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java:
##########
@@ -356,22 +379,25 @@ public void close() throws StorageContainerException {
       // Second sync should be a very light operation as sync has already
       // been done outside the lock.
       flushAndSyncDB();
-      updateContainerData(containerData::closeContainer);
+      updateContainerData(closer);
       clearPendingPutBlockCache();
     } finally {
       writeUnlock();
     }
-    LOG.info("Container {} is closed with bcsId {}.",
-        containerData.getContainerID(),
-        containerData.getBlockCommitSequenceId());
   }
 
-  @Override
-  public void updateDataScanTimestamp(Instant time)
+  /**
+   * For db-per-volume schemas, we don't flush and sync the whole db
+   * on closing of a single container.
+   * @param closer
+   * @throws StorageContainerException
+   */
+  private void closeWithoutFlush(Runnable closer)

Review Comment:
   Sure, that's a good idea.



-- 
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 diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java:
##########
@@ -271,6 +271,15 @@ List<? extends KeyValue<KEY, VALUE>> getSequentialRangeKVs(KEY startKey,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException;
 
+  /**
+   * Delete all keys with a prefix.
+   * @param batch
+   * @param prefix
+   * @return
+   */
+  void deleteAllWithPrefix(BatchOperation batch, KEY prefix)

Review Comment:
   deleteAllWithPrefix -> deleteWithBatch



-- 
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 diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java:
##########
@@ -127,6 +131,7 @@ public void setUp() throws Exception {
     HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
         .getAbsolutePath()).conf(CONF).datanodeUuid(datanodeId
         .toString()).build();
+    StorageVolumeUtil.checkVolume(hddsVolume, scmId, scmId, CONF, null, null);

Review Comment:
   Use createDbInstancesForTestIfNeeded instead? 



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

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

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


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


[GitHub] [ozone] guihecheng commented on a diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java:
##########
@@ -74,4 +75,15 @@ public BlockIterator<BlockData> getBlockIterator(long containerID,
         getBlockDataTableWithIterator()
             .iterator(getContainerKeyPrefix(containerID)), filter);
   }
+
+  public void dropAllWithPrefix(long containerID) throws IOException {

Review Comment:
   OK.



-- 
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 diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java:
##########
@@ -73,23 +76,28 @@ public class TestBlockManagerImpl {
   private BlockID blockID1;
 
   private final ContainerLayoutVersion layout;
+  private final String schemaVersion;
 
-  public TestBlockManagerImpl(ContainerLayoutVersion layout) {
-    this.layout = layout;
+  public TestBlockManagerImpl(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    this.config = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, config);
   }
 
   @Parameterized.Parameters
   public static Iterable<Object[]> parameters() {
-    return ContainerLayoutTestInfo.containerLayoutParameters();
+    return ContainerTestVersionInfo.versionParameters();
   }
 
   @Before
   public void setUp() throws Exception {
-    config = new OzoneConfiguration();
     UUID datanodeId = UUID.randomUUID();
     HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
         .getAbsolutePath()).conf(config).datanodeUuid(datanodeId
         .toString()).build();
+    StorageVolumeUtil.checkVolume(hddsVolume, scmId, scmId, config,
+        null, null);

Review Comment:
   Shall we call createDbInstancesForTestIfNeeded instead?



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

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

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


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


[GitHub] [ozone] ChenSammi commented on a diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java:
##########
@@ -271,6 +271,15 @@ List<? extends KeyValue<KEY, VALUE>> getSequentialRangeKVs(KEY startKey,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException;
 
+  /**
+   * Delete all keys with a prefix.

Review Comment:
   Deletes all keys with the specified prefix from the metadata store as part of a batch operation.



-- 
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] guihecheng commented on a diff in pull request #3346: HDDS-6542. [Merge rocksdb in datanode] KeyValueContainer operation adaptation for schema v3 containers.

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


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java:
##########
@@ -237,4 +237,21 @@ public static void verifyBCSId(KeyValueContainer container, BlockID blockID)
               + containerBCSId + ".", UNKNOWN_BCSID);
     }
   }
+
+  /**
+   * Remove container KV metadata from per-disk db store.
+   * @param containerData
+   * @param conf
+   * @throws IOException
+   */
+  public static void dropAllFromDB(KeyValueContainerData containerData,

Review Comment:
   For 2, actually it is used when the disk failed, then the db handler should be removed.



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

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