Merge remote-tracking branch 'origin/master' into HDDS-3630
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java
index 4217055..d2b07ef 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java
@@ -33,7 +33,9 @@
       "families)"),
   SCM_HA(2, "Storage Container Manager HA"),
   ERASURE_CODED_STORAGE_SUPPORT(3, "Ozone version with built in support for"
-      + " Erasure Coded block data storage.");
+      + " Erasure Coded block data storage."),
+  DATANODE_SCHEMA_V3(4, "Datanode RocksDB Schema Version 3 (one rocksdb " +
+      "per disk)");
 
   //////////////////////////////  //////////////////////////////
 
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index c9ceccc..1a47ad9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -316,6 +316,11 @@
   public static final String
       HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT = "1GB";
 
+  // Specifying the dedicated volumes for per-disk db instances.
+  // For container schema v3 only.
+  public static final String HDDS_DATANODE_CONTAINER_DB_DIR =
+      "hdds.datanode.container.db.dir";
+
   public static final String OZONE_SECURITY_ENABLED_KEY =
       "ozone.security.enabled";
   public static final boolean OZONE_SECURITY_ENABLED_DEFAULT = false;
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index fcbcff3..4fc0adf 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -135,6 +135,7 @@
   public static final String SCM_DB_NAME = "scm.db";
   public static final String OM_DB_BACKUP_PREFIX = "om.db.backup.";
   public static final String SCM_DB_BACKUP_PREFIX = "scm.db.backup.";
+  public static final String CONTAINER_DB_NAME = "container.db";
 
   public static final String STORAGE_DIR_CHUNKS = "chunks";
   public static final String OZONE_DB_CHECKPOINT_REQUEST_FLUSH =
@@ -276,9 +277,12 @@
   // V2: Metadata, block data, and delete transactions in their own
   // column families.
   public static final String SCHEMA_V2 = "2";
+  // V3: Column families definitions are close to V2,
+  // but have containerID as key prefixes.
+  public static final String SCHEMA_V3 = "3";
 
   public static final String[] SCHEMA_VERSIONS =
-      new String[] {SCHEMA_V1, SCHEMA_V2};
+      new String[] {SCHEMA_V1, SCHEMA_V2, SCHEMA_V3};
 
   // Supported store types.
   public static final String OZONE = "ozone";
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/AbstractLayoutVersionManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/AbstractLayoutVersionManager.java
index 2ce6e02..ccf7c1f 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/AbstractLayoutVersionManager.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/AbstractLayoutVersionManager.java
@@ -37,6 +37,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.management.ObjectName;
+
 /**
  * Layout Version Manager containing generic method implementations.
  */
@@ -58,6 +60,7 @@
   // Note that MLV may have been incremented during the upgrade
   // by the time the value is read/used.
   private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private ObjectName mBean;
 
   protected void init(int version, T[] lfs) throws IOException {
     lock.writeLock().lock();
@@ -81,7 +84,7 @@
           mlvFeature, mlvFeature.layoutVersion(),
           slvFeature, slvFeature.layoutVersion());
 
-      MBeans.register("LayoutVersionManager",
+      mBean = MBeans.register("LayoutVersionManager",
           getClass().getSimpleName(), this);
     } finally {
       lock.writeLock().unlock();
@@ -215,4 +218,11 @@
       lock.readLock().unlock();
     }
   }
+
+  public void close() {
+    if (mBean != null) {
+      MBeans.unregister(mBean);
+      mBean = null;
+    }
+  }
 }
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 6bdad25..21c580b 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -145,6 +145,20 @@
     </description>
   </property>
   <property>
+    <name>hdds.datanode.container.db.dir</name>
+    <value/>
+    <tag>OZONE, CONTAINER, STORAGE, MANAGEMENT</tag>
+    <description>Determines where the per-disk rocksdb instances will be
+      stored. This setting is optional. If unspecified, then rocksdb instances
+      are stored on the same disk as HDDS data.
+      The directories should be tagged with corresponding storage types
+      ([SSD]/[DISK]/[ARCHIVE]/[RAM_DISK]) for storage policies. The default
+      storage type will be DISK if the directory does not have a storage type
+      tagged explicitly. Ideally, this should be mapped to a fast disk
+      like an SSD.
+    </description>
+  </property>
+  <property>
     <name>hdds.datanode.dir.du.reserved</name>
     <value/>
     <tag>OZONE, CONTAINER, STORAGE, MANAGEMENT</tag>
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
index de53606..c4962b0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
@@ -57,7 +57,7 @@
 import org.apache.hadoop.ozone.container.common.DatanodeLayoutStorage;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
-import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
@@ -322,8 +322,8 @@
 
     for (Map.Entry<String, StorageVolume> entry : volumeMap.entrySet()) {
       HddsVolume hddsVolume = (HddsVolume) entry.getValue();
-      boolean result = HddsVolumeUtil.checkVolume(hddsVolume, clusterId,
-          clusterId, conf, LOG);
+      boolean result = StorageVolumeUtil.checkVolume(hddsVolume, clusterId,
+          clusterId, conf, LOG, null);
       if (!result) {
         volumeSet.failVolume(hddsVolume.getHddsRootDir().getPath());
       }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/DBHandle.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/DBHandle.java
new file mode 100644
index 0000000..839a112
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/DBHandle.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.interfaces;
+
+import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
+
+import java.io.Closeable;
+
+/**
+ * DB handle abstract class.
+ */
+public abstract class DBHandle implements Closeable {
+
+  private final DatanodeStore store;
+  private final String containerDBPath;
+
+  public DBHandle(DatanodeStore store, String containerDBPath) {
+    this.store = store;
+    this.containerDBPath = containerDBPath;
+  }
+
+  public DatanodeStore getStore() {
+    return this.store;
+  }
+
+  public String getContainerDBPath() {
+    return this.containerDBPath;
+  }
+
+  public boolean cleanup() {
+    return true;
+  }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java
index 24df9f5..3a11c53 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java
@@ -45,6 +45,8 @@
       "hdds.datanode.failed.data.volumes.tolerated";
   public static final String FAILED_METADATA_VOLUMES_TOLERATED_KEY =
       "hdds.datanode.failed.metadata.volumes.tolerated";
+  public static final String FAILED_DB_VOLUMES_TOLERATED_KEY =
+      "hdds.datanode.failed.db.volumes.tolerated";
   public static final String DISK_CHECK_MIN_GAP_KEY =
       "hdds.datanode.disk.check.min.gap";
   public static final String DISK_CHECK_TIMEOUT_KEY =
@@ -52,6 +54,8 @@
 
   public static final String WAIT_ON_ALL_FOLLOWERS =
       "hdds.datanode.wait.on.all.followers";
+  public static final String CONTAINER_SCHEMA_V3_ENABLED =
+      "hdds.datanode.container.schema.v3.enabled";
 
   static final boolean CHUNK_DATA_VALIDATION_CHECK_DEFAULT = false;
 
@@ -67,6 +71,8 @@
   static final long DISK_CHECK_TIMEOUT_DEFAULT =
       Duration.ofMinutes(10).toMillis();
 
+  static final boolean CONTAINER_SCHEMA_V3_ENABLED_DEFAULT = false;
+
   /**
    * Number of threads per volume that Datanode will use for chunk read.
    */
@@ -195,6 +201,17 @@
   )
   private int failedMetadataVolumesTolerated = FAILED_VOLUMES_TOLERATED_DEFAULT;
 
+  @Config(key = "failed.db.volumes.tolerated",
+      defaultValue = "-1",
+      type = ConfigType.INT,
+      tags = { DATANODE },
+      description = "The number of db volumes that are allowed to fail "
+          + "before a datanode stops offering service. "
+          + "Config this to -1 means unlimited, but we should have "
+          + "at least one good volume left."
+  )
+  private int failedDbVolumesTolerated = FAILED_VOLUMES_TOLERATED_DEFAULT;
+
   @Config(key = "disk.check.min.gap",
       defaultValue = "15m",
       type = ConfigType.TIME,
@@ -245,6 +262,24 @@
     this.waitOnAllFollowers = val;
   }
 
+  @Config(key = "container.schema.v3.enabled",
+      defaultValue = "false",
+      type = ConfigType.BOOLEAN,
+      tags = { DATANODE },
+      description = "Enable use of container schema v3(one rocksdb per disk)."
+  )
+  private boolean containerSchemaV3Enabled =
+      CONTAINER_SCHEMA_V3_ENABLED_DEFAULT;
+
+  @Config(key = "container.schema.v3.key.separator",
+      defaultValue = "|",
+      type = ConfigType.STRING,
+      tags = { DATANODE },
+      description = "The default separator between Container ID and container" +
+           " meta key name."
+  )
+  private String containerSchemaV3KeySeparator = "|";
+
   @PostConstruct
   public void validate() {
     if (containerDeleteThreads < 1) {
@@ -277,6 +312,13 @@
       failedMetadataVolumesTolerated = FAILED_VOLUMES_TOLERATED_DEFAULT;
     }
 
+    if (failedDbVolumesTolerated < -1) {
+      LOG.warn(FAILED_DB_VOLUMES_TOLERATED_KEY +
+              "must be greater than -1 and was set to {}. Defaulting to {}",
+          failedDbVolumesTolerated, FAILED_VOLUMES_TOLERATED_DEFAULT);
+      failedDbVolumesTolerated = FAILED_VOLUMES_TOLERATED_DEFAULT;
+    }
+
     if (diskCheckMinGap < 0) {
       LOG.warn(DISK_CHECK_MIN_GAP_KEY +
               " must be greater than zero and was set to {}. Defaulting to {}",
@@ -325,6 +367,14 @@
     this.failedMetadataVolumesTolerated = failedVolumesTolerated;
   }
 
+  public int getFailedDbVolumesTolerated() {
+    return failedDbVolumesTolerated;
+  }
+
+  public void setFailedDbVolumesTolerated(int failedVolumesTolerated) {
+    this.failedDbVolumesTolerated = failedVolumesTolerated;
+  }
+
   public Duration getDiskCheckMinGap() {
     return Duration.ofMillis(diskCheckMinGap);
   }
@@ -372,4 +422,20 @@
   public int getNumReadThreadPerVolume() {
     return numReadThreadPerVolume;
   }
+
+  public boolean getContainerSchemaV3Enabled() {
+    return this.containerSchemaV3Enabled;
+  }
+
+  public void setContainerSchemaV3Enabled(boolean containerSchemaV3Enabled) {
+    this.containerSchemaV3Enabled = containerSchemaV3Enabled;
+  }
+
+  public String getContainerSchemaV3KeySeparator() {
+    return this.containerSchemaV3KeySeparator;
+  }
+
+  public void setContainerSchemaV3KeySeparator(String separator) {
+    this.containerSchemaV3KeySeparator = separator;
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index 0089522..ba21e57 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -327,8 +327,8 @@
   public void handleFatalVolumeFailures() {
     LOG.error("DatanodeStateMachine Shutdown due to too many bad volumes, "
         + "check " + DatanodeConfiguration.FAILED_DATA_VOLUMES_TOLERATED_KEY
-        + " and "
-        + DatanodeConfiguration.FAILED_METADATA_VOLUMES_TOLERATED_KEY);
+        + " and " + DatanodeConfiguration.FAILED_METADATA_VOLUMES_TOLERATED_KEY
+        + " and " + DatanodeConfiguration.FAILED_DB_VOLUMES_TOLERATED_KEY);
     hddsDatanodeStopService.stopService();
   }
 
@@ -369,6 +369,9 @@
     if (cmdProcessThread != null) {
       cmdProcessThread.interrupt();
     }
+    if (layoutVersionManager != null) {
+      layoutVersionManager.close();
+    }
     context.setState(DatanodeStates.getLastState());
     replicationSupervisorMetrics.unRegister();
     executorService.shutdown();
@@ -689,4 +692,8 @@
   public UpgradeFinalizer<DatanodeStateMachine> getUpgradeFinalizer() {
     return upgradeFinalizer;
   }
+
+  public ConfigurationSource getConf() {
+    return conf;
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
index 1ec8329..8e95017 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
@@ -31,20 +31,19 @@
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
 import org.apache.hadoop.hdds.utils.db.Table;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfoList;
 import org.apache.hadoop.ozone.container.common.helpers
     .DeletedContainerBlocksSummary;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.statemachine
     .SCMConnectionManager;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
-import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
-import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaTwoImpl;
+import org.apache.hadoop.ozone.container.metadata.DeleteTransactionStore;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.protocol.commands.CommandStatus;
 import org.apache.hadoop.ozone.protocol.commands.DeleteBlockCommandStatus;
@@ -53,7 +52,6 @@
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,6 +70,7 @@
     .Result.CONTAINER_NOT_FOUND;
 import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V1;
 import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V2;
+import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V3;
 
 /**
  * Handle block deletion commands.
@@ -230,10 +229,12 @@
             } else if (containerData.getSchemaVersion().equals(SCHEMA_V2)) {
               markBlocksForDeletionSchemaV2(containerData, tx,
                   newDeletionBlocks, tx.getTxID());
+            } else if (containerData.getSchemaVersion().equals(SCHEMA_V3)) {
+              markBlocksForDeletionSchemaV3(containerData, tx,
+                  newDeletionBlocks, tx.getTxID());
             } else {
               throw new UnsupportedOperationException(
-                  "Only schema version 1 and schema version 2 are "
-                      + "supported.");
+                  "Only schema version 1,2,3 are supported.");
             }
           } finally {
             cont.writeUnlock();
@@ -327,6 +328,34 @@
     }
   }
 
+  private void markBlocksForDeletionSchemaV3(
+      KeyValueContainerData containerData, DeletedBlocksTransaction delTX,
+      int newDeletionBlocks, long txnID)
+      throws IOException {
+    DeletionMarker schemaV3Marker = (table, batch, tid, txn) -> {
+      Table<String, DeletedBlocksTransaction> delTxTable =
+          (Table<String, DeletedBlocksTransaction>) table;
+      delTxTable.putWithBatch(batch, containerData.deleteTxnKey(tid), txn);
+    };
+
+    markBlocksForDeletionTransaction(containerData, delTX, newDeletionBlocks,
+        txnID, schemaV3Marker);
+  }
+
+  private void markBlocksForDeletionSchemaV2(
+      KeyValueContainerData containerData, DeletedBlocksTransaction delTX,
+      int newDeletionBlocks, long txnID)
+      throws IOException {
+    DeletionMarker schemaV2Marker = (table, batch, tid, txn) -> {
+      Table<Long, DeletedBlocksTransaction> delTxTable =
+          (Table<Long, DeletedBlocksTransaction>) table;
+      delTxTable.putWithBatch(batch, tid, txn);
+    };
+
+    markBlocksForDeletionTransaction(containerData, delTX, newDeletionBlocks,
+        txnID, schemaV2Marker);
+  }
+
   /**
    * Move a bunch of blocks from a container to deleting state. This is a meta
    * update, the actual deletes happen in async mode.
@@ -335,24 +364,22 @@
    * @param delTX a block deletion transaction.
    * @throws IOException if I/O error occurs.
    */
-
-  private void markBlocksForDeletionSchemaV2(
+  private void markBlocksForDeletionTransaction(
       KeyValueContainerData containerData, DeletedBlocksTransaction delTX,
-      int newDeletionBlocks, long txnID) throws IOException {
+      int newDeletionBlocks, long txnID, DeletionMarker marker)
+      throws IOException {
     long containerId = delTX.getContainerID();
     if (!isTxnIdValid(containerId, containerData, delTX)) {
       return;
     }
-    try (ReferenceCountedDB containerDB = BlockUtils
-        .getDB(containerData, conf)) {
-      DatanodeStore ds = containerDB.getStore();
-      DatanodeStoreSchemaTwoImpl dnStoreTwoImpl =
-          (DatanodeStoreSchemaTwoImpl) ds;
-      Table<Long, DeletedBlocksTransaction> delTxTable =
-          dnStoreTwoImpl.getDeleteTransactionTable();
+    try (DBHandle containerDB = BlockUtils.getDB(containerData, conf)) {
+      DeleteTransactionStore<?> store =
+          (DeleteTransactionStore<?>) containerDB.getStore();
+      Table<?, DeletedBlocksTransaction> delTxTable =
+          store.getDeleteTransactionTable();
       try (BatchOperation batch = containerDB.getStore().getBatchHandler()
           .initBatchOperation()) {
-        delTxTable.putWithBatch(batch, txnID, delTX);
+        marker.apply(delTxTable, batch, txnID, delTX);
         newDeletionBlocks += delTX.getLocalIDList().size();
         updateMetaData(containerData, delTX, newDeletionBlocks, containerDB,
             batch);
@@ -369,8 +396,7 @@
       return;
     }
     int newDeletionBlocks = 0;
-    try (ReferenceCountedDB containerDB = BlockUtils
-        .getDB(containerData, conf)) {
+    try (DBHandle containerDB = BlockUtils.getDB(containerData, conf)) {
       Table<String, BlockData> blockDataTable =
           containerDB.getStore().getBlockDataTable();
       Table<String, ChunkInfoList> deletedBlocksTable =
@@ -379,16 +405,15 @@
       try (BatchOperation batch = containerDB.getStore().getBatchHandler()
           .initBatchOperation()) {
         for (Long blkLong : delTX.getLocalIDList()) {
-          String blk = blkLong.toString();
+          String blk = containerData.blockKey(blkLong);
           BlockData blkInfo = blockDataTable.get(blk);
           if (blkInfo != null) {
-            String deletingKey = OzoneConsts.DELETING_KEY_PREFIX + blk;
+            String deletingKey = containerData.deletingBlockKey(blkLong);
             if (blockDataTable.get(deletingKey) != null
                 || deletedBlocksTable.get(blk) != null) {
               if (LOG.isDebugEnabled()) {
-                LOG.debug(String.format(
-                    "Ignoring delete for block %s in container %d."
-                        + " Entry already added.", blk, containerId));
+                LOG.debug("Ignoring delete for block {} in container {}."
+                        + " Entry already added.", blkLong, containerId);
               }
               continue;
             }
@@ -399,12 +424,12 @@
             newDeletionBlocks++;
             if (LOG.isDebugEnabled()) {
               LOG.debug("Transited Block {} to DELETING state in container {}",
-                  blk, containerId);
+                  blkLong, containerId);
             }
           } else {
             if (LOG.isDebugEnabled()) {
               LOG.debug("Block {} not found or already under deletion in"
-                  + " container {}, skip deleting it.", blk, containerId);
+                  + " container {}, skip deleting it.", blkLong, containerId);
             }
           }
         }
@@ -423,7 +448,7 @@
 
   private void updateMetaData(KeyValueContainerData containerData,
       DeletedBlocksTransaction delTX, int newDeletionBlocks,
-      ReferenceCountedDB containerDB, BatchOperation batchOperation)
+      DBHandle containerDB, BatchOperation batchOperation)
       throws IOException {
     if (newDeletionBlocks > 0) {
       // Finally commit the DB counters.
@@ -435,14 +460,15 @@
       if (delTX.getTxID() > containerData.getDeleteTransactionId()) {
         // Update in DB pending delete key count and delete transaction ID.
         metadataTable
-            .putWithBatch(batchOperation, OzoneConsts.DELETE_TRANSACTION_KEY,
+            .putWithBatch(batchOperation, containerData.latestDeleteTxnKey(),
                 delTX.getTxID());
       }
 
       long pendingDeleteBlocks =
           containerData.getNumPendingDeletionBlocks() + newDeletionBlocks;
       metadataTable
-          .putWithBatch(batchOperation, OzoneConsts.PENDING_DELETE_BLOCK_COUNT,
+          .putWithBatch(batchOperation,
+              containerData.pendingDeleteBlockCountKey(),
               pendingDeleteBlocks);
 
       // update pending deletion blocks count and delete transaction ID in
@@ -513,4 +539,10 @@
       }
     }
   }
+
+  private interface DeletionMarker {
+    void apply(Table<?, DeletedBlocksTransaction> deleteTxnsTable,
+        BatchOperation batch, long txnID, DeletedBlocksTransaction delTX)
+        throws IOException;
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
index d80d1e5..f9b0d88 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
@@ -17,15 +17,13 @@
 package org.apache.hadoop.ozone.container.common.states.endpoint;
 
 import java.io.IOException;
-import java.util.Map;
 import java.util.concurrent.Callable;
 
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
-import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
-import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
@@ -78,36 +76,16 @@
           String scmId = response.getValue(OzoneConsts.SCM_ID);
           String clusterId = response.getValue(OzoneConsts.CLUSTER_ID);
 
-          // Check volumes
-          MutableVolumeSet volumeSet = ozoneContainer.getVolumeSet();
-          volumeSet.writeLock();
-          try {
-            Map<String, StorageVolume> volumeMap = volumeSet.getVolumeMap();
+          Preconditions.checkNotNull(scmId,
+              "Reply from SCM: scmId cannot be null");
+          Preconditions.checkNotNull(clusterId,
+              "Reply from SCM: clusterId cannot be null");
 
-            Preconditions.checkNotNull(scmId,
-                "Reply from SCM: scmId cannot be null");
-            Preconditions.checkNotNull(clusterId,
-                "Reply from SCM: clusterId cannot be null");
+          // Check DbVolumes, format DbVolume at first register time.
+          checkVolumeSet(ozoneContainer.getDbVolumeSet(), scmId, clusterId);
 
-            // If version file does not exist
-            // create version file and also set scm ID or cluster ID.
-            for (Map.Entry<String, StorageVolume> entry
-                : volumeMap.entrySet()) {
-              StorageVolume volume = entry.getValue();
-              boolean result = HddsVolumeUtil.checkVolume((HddsVolume) volume,
-                  scmId, clusterId, configuration, LOG);
-              if (!result) {
-                volumeSet.failVolume(volume.getStorageDir().getPath());
-              }
-            }
-            if (volumeSet.getVolumesList().size() == 0) {
-              // All volumes are in inconsistent state
-              throw new DiskOutOfSpaceException(
-                  "All configured Volumes are in Inconsistent State");
-            }
-          } finally {
-            volumeSet.writeUnlock();
-          }
+          // Check HddsVolumes
+          checkVolumeSet(ozoneContainer.getVolumeSet(), scmId, clusterId);
 
           // Start the container services after getting the version information
           ozoneContainer.start(clusterId);
@@ -129,4 +107,32 @@
     }
     return rpcEndPoint.getState();
   }
+
+  private void checkVolumeSet(MutableVolumeSet volumeSet,
+      String scmId, String clusterId) throws DiskOutOfSpaceException {
+    if (volumeSet == null) {
+      return;
+    }
+
+    volumeSet.writeLock();
+    try {
+      // If version file does not exist
+      // create version file and also set scm ID or cluster ID.
+      for (StorageVolume volume : volumeSet.getVolumeMap().values()) {
+        boolean result = StorageVolumeUtil.checkVolume(volume,
+            scmId, clusterId, configuration, LOG,
+            ozoneContainer.getDbVolumeSet());
+        if (!result) {
+          volumeSet.failVolume(volume.getStorageDir().getPath());
+        }
+      }
+      if (volumeSet.getVolumesList().size() == 0) {
+        // All volumes are in inconsistent state
+        throw new DiskOutOfSpaceException(
+            "All configured Volumes are in Inconsistent State");
+      }
+    } finally {
+      volumeSet.writeUnlock();
+    }
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
index af0958a..1551836 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
@@ -156,7 +156,7 @@
 
       try {
         long start = Time.monotonicNow();
-        DatanodeStore store = BlockUtils.getUncachedDatanodeStore(containerID,
+        DatanodeStore store = BlockUtils.getUncachedDatanodeStore(
             containerDBPath, schemaVersion, conf, false);
         db = new ReferenceCountedDB(store, containerDBPath);
         metrics.incDbOpenLatency(Time.monotonicNow() - start);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/DatanodeStoreCache.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/DatanodeStoreCache.java
new file mode 100644
index 0000000..0f7baa6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/DatanodeStoreCache.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Cache for all per-disk DB handles under schema v3.
+ */
+public final class DatanodeStoreCache {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DatanodeStoreCache.class);
+
+  /**
+   * Use container db absolute path as key.
+   */
+  private final Map<String, RawDB> datanodeStoreMap;
+
+  private static DatanodeStoreCache cache;
+
+  private DatanodeStoreCache() {
+    datanodeStoreMap = new ConcurrentHashMap<>();
+  }
+
+  public static synchronized DatanodeStoreCache getInstance() {
+    if (cache == null) {
+      cache = new DatanodeStoreCache();
+    }
+    return cache;
+  }
+
+  public void addDB(String containerDBPath, RawDB db) {
+    datanodeStoreMap.putIfAbsent(containerDBPath, db);
+  }
+
+  public RawDB getDB(String containerDBPath) {
+    return datanodeStoreMap.get(containerDBPath);
+  }
+
+  public void removeDB(String containerDBPath) {
+    RawDB db = datanodeStoreMap.remove(containerDBPath);
+    if (db == null) {
+      LOG.debug("DB {} already removed", containerDBPath);
+      return;
+    }
+
+    try {
+      db.getStore().stop();
+    } catch (Exception e) {
+      LOG.error("Stop DatanodeStore: {} failed", containerDBPath, e);
+    }
+  }
+
+  public void shutdownCache() {
+    for (Map.Entry<String, RawDB> entry : datanodeStoreMap.entrySet()) {
+      try {
+        entry.getValue().getStore().stop();
+      } catch (Exception e) {
+        LOG.warn("Stop DatanodeStore: {} failed", entry.getKey(), e);
+      }
+    }
+    datanodeStoreMap.clear();
+  }
+
+  public int size() {
+    return datanodeStoreMap.size();
+  }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
index 6a38080..0e1414f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
@@ -18,21 +18,22 @@
 
 package org.apache.hadoop.ozone.container.common.utils;
 
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
-import org.apache.hadoop.ozone.container.common.HDDSVolumeLayoutVersion;
+import org.apache.hadoop.ozone.container.common.volume.DbVolume;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
-import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
-import org.apache.hadoop.util.Time;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
+import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
 import org.slf4j.Logger;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Properties;
-import java.util.UUID;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil.onFailure;
 
 /**
  * A util class for {@link HddsVolume}.
@@ -43,17 +44,6 @@
   private HddsVolumeUtil() {
   }
 
-  private static final String VERSION_FILE   = "VERSION";
-  private static final String STORAGE_ID_PREFIX = "DS-";
-
-  public static File getVersionFile(File rootDir) {
-    return new File(rootDir, VERSION_FILE);
-  }
-
-  public static String generateUuid() {
-    return STORAGE_ID_PREFIX + UUID.randomUUID();
-  }
-
   /**
    * Get hddsRoot from volume root. If volumeRoot points to hddsRoot, it is
    * returned as is.
@@ -71,167 +61,65 @@
   }
 
   /**
-   * Returns storageID if it is valid. Throws an exception otherwise.
+   * Initialize db instance, rocksdb will load the existing instance
+   * if present and format a new one if not.
+   * @param containerDBPath
+   * @param conf
+   * @throws IOException
    */
-  @VisibleForTesting
-  public static String getStorageID(Properties props, File versionFile)
-      throws InconsistentStorageStateException {
-    return getProperty(props, OzoneConsts.STORAGE_ID, versionFile);
+  public static void initPerDiskDBStore(String containerDBPath,
+      ConfigurationSource conf) throws IOException {
+    DatanodeStore store = BlockUtils.getUncachedDatanodeStore(containerDBPath,
+        OzoneConsts.SCHEMA_V3, conf, false);
+    BlockUtils.addDB(store, containerDBPath, conf, OzoneConsts.SCHEMA_V3);
   }
 
   /**
-   * Returns clusterID if it is valid. It should match the clusterID from the
-   * Datanode. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static String getClusterID(Properties props, File versionFile,
-      String clusterID) throws InconsistentStorageStateException {
-    String cid = getProperty(props, OzoneConsts.CLUSTER_ID, versionFile);
-
-    if (clusterID == null) {
-      return cid;
-    }
-    if (!clusterID.equals(cid)) {
-      throw new InconsistentStorageStateException("Mismatched " +
-          "ClusterIDs. Version File : " + versionFile + " has clusterID: " +
-          cid + " and Datanode has clusterID: " + clusterID);
-    }
-    return cid;
-  }
-
-  /**
-   * Returns datanodeUuid if it is valid. It should match the UUID of the
-   * Datanode. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static String getDatanodeUUID(Properties props, File versionFile,
-      String datanodeUuid)
-      throws InconsistentStorageStateException {
-    String datanodeID = getProperty(props, OzoneConsts.DATANODE_UUID,
-        versionFile);
-
-    if (datanodeUuid != null && !datanodeUuid.equals(datanodeID)) {
-      throw new InconsistentStorageStateException("Mismatched " +
-          "DatanodeUUIDs. Version File : " + versionFile + " has datanodeUuid: "
-          + datanodeID + " and Datanode has datanodeUuid: " + datanodeUuid);
-    }
-    return datanodeID;
-  }
-
-  /**
-   * Returns creationTime if it is valid. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static long getCreationTime(Properties props, File versionFile)
-      throws InconsistentStorageStateException {
-    String cTimeStr = getProperty(props, OzoneConsts.CTIME, versionFile);
-
-    long cTime = Long.parseLong(cTimeStr);
-    long currentTime = Time.now();
-    if (cTime > currentTime || cTime < 0) {
-      throw new InconsistentStorageStateException("Invalid Creation time in " +
-          "Version File : " + versionFile + " - " + cTime + ". Current system" +
-          " time is " + currentTime);
-    }
-    return cTime;
-  }
-
-  /**
-   * Returns layOutVersion if it is valid. Throws an exception otherwise.
-   */
-  @VisibleForTesting
-  public static int getLayOutVersion(Properties props, File versionFile) throws
-      InconsistentStorageStateException {
-    String lvStr = getProperty(props, OzoneConsts.LAYOUTVERSION, versionFile);
-
-    int lv = Integer.parseInt(lvStr);
-    if (HDDSVolumeLayoutVersion.getLatestVersion().getVersion() != lv) {
-      throw new InconsistentStorageStateException("Invalid layOutVersion. " +
-          "Version file has layOutVersion as " + lv + " and latest Datanode " +
-          "layOutVersion is " +
-          HDDSVolumeLayoutVersion.getLatestVersion().getVersion());
-    }
-    return lv;
-  }
-
-  public static String getProperty(
-      Properties props, String propName, File
-      versionFile
-  )
-      throws InconsistentStorageStateException {
-    String value = props.getProperty(propName);
-    if (StringUtils.isBlank(value)) {
-      throw new InconsistentStorageStateException("Invalid " + propName +
-          ". Version File : " + versionFile + " has null or empty " + propName);
-    }
-    return value;
-  }
-
-  /**
-   * Check Volume is in consistent state or not.
-   * Prior to SCM HA, volumes used the format {@code <volume>/hdds/<scm-id>}.
-   * Post SCM HA, new volumes will use the format {@code <volume>/hdds/<cluster
-   * -id>}.
-   * Existing volumes using SCM ID would have been reformatted to have {@code
-   * <volume>/hdds/<cluster-id>} as a symlink pointing to {@code <volume
-   * >/hdds/<scm-id>}.
-   *
-   * @param hddsVolume
-   * @param clusterId
+   * Load already formatted db instances for all HddsVolumes.
+   * @param hddsVolumeSet
+   * @param dbVolumeSet
    * @param logger
-   * @return true - if volume is in consistent state, otherwise false.
    */
-  public static boolean checkVolume(HddsVolume hddsVolume, String scmId,
-      String clusterId, ConfigurationSource conf, Logger logger) {
-    File hddsRoot = hddsVolume.getHddsRootDir();
-    String volumeRoot = hddsRoot.getPath();
-    File clusterDir = new File(hddsRoot, clusterId);
+  public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
+      MutableVolumeSet dbVolumeSet, Logger logger) {
+    // Scan subdirs under the db volumes and build a one-to-one map
+    // between each HddsVolume -> DbVolume.
+    mapDbVolumesToDataVolumesIfNeeded(hddsVolumeSet, dbVolumeSet);
 
-    try {
-      hddsVolume.format(clusterId);
-    } catch (IOException ex) {
-      logger.error("Error during formatting volume {}.",
-          volumeRoot, ex);
-      return false;
+    for (HddsVolume volume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      try {
+        volume.loadDbStore();
+      } catch (IOException e) {
+        onFailure(volume);
+        if (logger != null) {
+          logger.error("Load db store for HddsVolume {} failed",
+              volume.getStorageDir().getAbsolutePath(), e);
+        }
+      }
+    }
+  }
+
+  private static void mapDbVolumesToDataVolumesIfNeeded(
+      MutableVolumeSet hddsVolumeSet, MutableVolumeSet dbVolumeSet) {
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      return;
     }
 
-    File[] hddsFiles = hddsRoot.listFiles();
+    List<HddsVolume> hddsVolumes = StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList());
+    List<DbVolume> dbVolumes = StorageVolumeUtil.getDbVolumesList(
+        dbVolumeSet.getVolumesList());
+    Map<String, DbVolume> globalDbVolumeMap = new HashMap<>();
 
-    if (hddsFiles == null) {
-      // This is the case for IOException, where listFiles returns null.
-      // So, we fail the volume.
-      return false;
-    } else if (hddsFiles.length == 1) {
-      // DN started for first time or this is a newly added volume.
-      // The one file is the version file.
-      // So we create cluster ID directory, or SCM ID directory if
-      // pre-finalized for SCM HA.
-      // Either the SCM ID or cluster ID will be used in naming the
-      // volume's subdirectory, depending on the datanode's layout version.
-      String id = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(conf,
-          scmId, clusterId);
-      File idDir = new File(hddsRoot, id);
-      if (!idDir.mkdir()) {
-        logger.error("Unable to create ID directory {} for datanode.", idDir);
-        return false;
-      }
-      return true;
-    } else if (hddsFiles.length == 2) {
-      // If we are finalized for SCM HA and there is no cluster ID directory,
-      // the volume may have been unhealthy during finalization and been
-      // skipped. Create cluster ID symlink now.
-      // Else, We are still pre-finalized.
-      // The existing directory should be left for backwards compatibility.
-      return VersionedDatanodeFeatures.ScmHA.
-          upgradeVolumeIfNeeded(hddsVolume, clusterId);
-    } else {
-      if (!clusterDir.exists()) {
-        logger.error("Volume {} is in an inconsistent state. {} files found " +
-            "but cluster ID directory {} does not exist.", volumeRoot,
-            hddsFiles.length, clusterDir);
-        return false;
-      }
-      return true;
-    }
+    // build a datanode global map of storageID -> dbVolume
+    dbVolumes.forEach(dbVolume ->
+        dbVolume.getHddsVolumeIDs().forEach(storageID ->
+            globalDbVolumeMap.put(storageID, dbVolume)));
+
+    // map each hddsVolume to a dbVolume
+    hddsVolumes.forEach(hddsVolume ->
+        hddsVolume.setDbVolume(globalDbVolumeMap.getOrDefault(
+            hddsVolume.getStorageID(), null)));
   }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/RawDB.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/RawDB.java
new file mode 100644
index 0000000..54849a6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/RawDB.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.utils;
+
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
+import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
+
+import java.io.IOException;
+
+/**
+ * Just a wrapper for DatanodeStore.
+ * This is for container schema v3 which has one rocksdb instance per disk.
+ */
+public class RawDB extends DBHandle {
+
+  public RawDB(DatanodeStore store, String containerDBPath) {
+    super(store, containerDBPath);
+  }
+
+  @Override
+  public void close() throws IOException {
+    // NOTE: intend to do nothing on close
+    // With schema v3, block operations on a single container should not
+    // close the whole db handle.
+    // Will close the low-level stores all together in a collection class.
+  }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ReferenceCountedDB.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ReferenceCountedDB.java
index 5fe61a8..3f85800 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ReferenceCountedDB.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ReferenceCountedDB.java
@@ -21,11 +21,11 @@
 import com.google.common.base.Preconditions;
 
 import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.Closeable;
 import java.util.concurrent.atomic.AtomicInteger;
 
 /**
@@ -35,17 +35,14 @@
  * from caller stack. When JDK9 StackWalker is available, we can switch to
  * StackWalker instead of new Exception().printStackTrace().
  */
-public class ReferenceCountedDB implements Closeable {
+public class ReferenceCountedDB extends DBHandle {
   private static final Logger LOG =
       LoggerFactory.getLogger(ReferenceCountedDB.class);
   private final AtomicInteger referenceCount;
-  private final DatanodeStore store;
-  private final String containerDBPath;
 
   public ReferenceCountedDB(DatanodeStore store, String containerDBPath) {
+    super(store, containerDBPath);
     this.referenceCount = new AtomicInteger(0);
-    this.store = store;
-    this.containerDBPath = containerDBPath;
   }
 
   public long getReferenceCount() {
@@ -55,7 +52,7 @@
   public void incrementReference() {
     this.referenceCount.incrementAndGet();
     if (LOG.isTraceEnabled()) {
-      LOG.trace("IncRef {} to refCnt {}, stackTrace: {}", containerDBPath,
+      LOG.trace("IncRef {} to refCnt {}, stackTrace: {}", getContainerDBPath(),
           referenceCount.get(), ExceptionUtils.getStackTrace(new Throwable()));
     }
   }
@@ -64,22 +61,22 @@
     int refCount = this.referenceCount.decrementAndGet();
     Preconditions.checkArgument(refCount >= 0, "refCount:", refCount);
     if (LOG.isTraceEnabled()) {
-      LOG.trace("DecRef {} to refCnt {}, stackTrace: {}", containerDBPath,
+      LOG.trace("DecRef {} to refCnt {}, stackTrace: {}", getContainerDBPath(),
           referenceCount.get(), ExceptionUtils.getStackTrace(new Throwable()));
     }
   }
 
   public boolean cleanup() {
-    if (referenceCount.get() == 0 && store != null) {
+    if (referenceCount.get() == 0 && getStore() != null) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Close {} refCnt {}", containerDBPath,
+        LOG.debug("Close {} refCnt {}", getContainerDBPath(),
             referenceCount.get());
       }
       try {
-        store.stop();
+        getStore().stop();
         return true;
       } catch (Exception e) {
-        LOG.error("Error closing DB. Container: " + containerDBPath, e);
+        LOG.error("Error closing DB. Container: " + getContainerDBPath(), e);
         return false;
       }
     } else {
@@ -87,10 +84,6 @@
     }
   }
 
-  public DatanodeStore getStore() {
-    return store;
-  }
-
   @Override
   public void close() {
     decrementReference();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/StorageVolumeUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/StorageVolumeUtil.java
index 104dbac..0050038 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/StorageVolumeUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/StorageVolumeUtil.java
@@ -18,12 +18,26 @@
 
 package org.apache.hadoop.ozone.container.common.utils;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.container.common.HDDSVolumeLayoutVersion;
+import org.apache.hadoop.ozone.container.common.volume.DbVolume;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
 
+import java.io.File;
+import java.io.IOException;
 import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
 import java.util.stream.Collectors;
 
 /**
@@ -31,6 +45,9 @@
  */
 public final class StorageVolumeUtil {
 
+  private static final String VERSION_FILE   = "VERSION";
+  private static final String STORAGE_ID_PREFIX = "DS-";
+
   private StorageVolumeUtil() {
   }
 
@@ -48,4 +65,189 @@
     return volumes.stream().
         map(v -> (HddsVolume) v).collect(Collectors.toList());
   }
+
+  public static List<DbVolume> getDbVolumesList(
+      List<StorageVolume> volumes) {
+    return volumes.stream().
+        map(v -> (DbVolume) v).collect(Collectors.toList());
+  }
+
+  public static File getVersionFile(File rootDir) {
+    return new File(rootDir, VERSION_FILE);
+  }
+
+  public static String generateUuid() {
+    return STORAGE_ID_PREFIX + UUID.randomUUID();
+  }
+
+  /**
+   * Returns storageID if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getStorageID(Properties props, File versionFile)
+      throws InconsistentStorageStateException {
+    return getProperty(props, OzoneConsts.STORAGE_ID, versionFile);
+  }
+
+  /**
+   * Returns clusterID if it is valid. It should match the clusterID from the
+   * Datanode. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getClusterID(Properties props, File versionFile,
+      String clusterID) throws InconsistentStorageStateException {
+    String cid = getProperty(props, OzoneConsts.CLUSTER_ID, versionFile);
+
+    if (clusterID == null) {
+      return cid;
+    }
+    if (!clusterID.equals(cid)) {
+      throw new InconsistentStorageStateException("Mismatched " +
+          "ClusterIDs. Version File : " + versionFile + " has clusterID: " +
+          cid + " and Datanode has clusterID: " + clusterID);
+    }
+    return cid;
+  }
+
+  /**
+   * Returns datanodeUuid if it is valid. It should match the UUID of the
+   * Datanode. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getDatanodeUUID(Properties props, File versionFile,
+      String datanodeUuid)
+      throws InconsistentStorageStateException {
+    String datanodeID = getProperty(props, OzoneConsts.DATANODE_UUID,
+        versionFile);
+
+    if (datanodeUuid != null && !datanodeUuid.equals(datanodeID)) {
+      throw new InconsistentStorageStateException("Mismatched " +
+          "DatanodeUUIDs. Version File : " + versionFile + " has datanodeUuid: "
+          + datanodeID + " and Datanode has datanodeUuid: " + datanodeUuid);
+    }
+    return datanodeID;
+  }
+
+  /**
+   * Returns creationTime if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static long getCreationTime(Properties props, File versionFile)
+      throws InconsistentStorageStateException {
+    String cTimeStr = getProperty(props, OzoneConsts.CTIME, versionFile);
+
+    long cTime = Long.parseLong(cTimeStr);
+    long currentTime = Time.now();
+    if (cTime > currentTime || cTime < 0) {
+      throw new InconsistentStorageStateException("Invalid Creation time in " +
+          "Version File : " + versionFile + " - " + cTime + ". Current system" +
+          " time is " + currentTime);
+    }
+    return cTime;
+  }
+
+  /**
+   * Returns layOutVersion if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static int getLayOutVersion(Properties props, File versionFile) throws
+      InconsistentStorageStateException {
+    String lvStr = getProperty(props, OzoneConsts.LAYOUTVERSION, versionFile);
+
+    int lv = Integer.parseInt(lvStr);
+    if (HDDSVolumeLayoutVersion.getLatestVersion().getVersion() != lv) {
+      throw new InconsistentStorageStateException("Invalid layOutVersion. " +
+          "Version file has layOutVersion as " + lv + " and latest Datanode " +
+          "layOutVersion is " +
+          HDDSVolumeLayoutVersion.getLatestVersion().getVersion());
+    }
+    return lv;
+  }
+
+  public static String getProperty(
+      Properties props, String propName, File
+      versionFile
+  )
+      throws InconsistentStorageStateException {
+    String value = props.getProperty(propName);
+    if (StringUtils.isBlank(value)) {
+      throw new InconsistentStorageStateException("Invalid " + propName +
+          ". Version File : " + versionFile + " has null or empty " + propName);
+    }
+    return value;
+  }
+
+  /**
+   * Check Volume is in consistent state or not.
+   * Prior to SCM HA, volumes used the format {@code <volume>/hdds/<scm-id>}.
+   * Post SCM HA, new volumes will use the format {@code <volume>/hdds/<cluster
+   * -id>}.
+   * Existing volumes using SCM ID would have been reformatted to have {@code
+   * <volume>/hdds/<cluster-id>} as a symlink pointing to {@code <volume
+   * >/hdds/<scm-id>}.
+   *
+   * @param volume
+   * @param scmId
+   * @param clusterId
+   * @param conf
+   * @param logger
+   * @param dbVolumeSet
+   * @return true - if volume is in consistent state, otherwise false.
+   */
+  public static boolean checkVolume(StorageVolume volume, String scmId,
+      String clusterId, ConfigurationSource conf, Logger logger,
+      MutableVolumeSet dbVolumeSet) {
+    File volumeRoot = volume.getStorageDir();
+    String volumeRootPath = volumeRoot.getPath();
+    File clusterDir = new File(volumeRoot, clusterId);
+
+    try {
+      volume.format(clusterId);
+    } catch (IOException ex) {
+      logger.error("Error during formatting volume {}.",
+          volumeRootPath, ex);
+      return false;
+    }
+
+    File[] rootFiles = volumeRoot.listFiles();
+
+    if (rootFiles == null) {
+      // This is the case for IOException, where listFiles returns null.
+      // So, we fail the volume.
+      return false;
+    } else if (rootFiles.length == 1) {
+      // DN started for first time or this is a newly added volume.
+      // The one file is the version file.
+      // So we create cluster ID directory, or SCM ID directory if
+      // pre-finalized for SCM HA.
+      // Either the SCM ID or cluster ID will be used in naming the
+      // volume's subdirectory, depending on the datanode's layout version.
+      String id = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(conf,
+          scmId, clusterId);
+      try {
+        volume.createWorkingDir(id, dbVolumeSet);
+      } catch (IOException e) {
+        logger.error("Prepare working dir failed for volume {}.",
+            volumeRootPath, e);
+        return false;
+      }
+      return true;
+    } else if (rootFiles.length == 2) {
+      // If we are finalized for SCM HA and there is no cluster ID directory,
+      // the volume may have been unhealthy during finalization and been
+      // skipped. Create cluster ID symlink now.
+      // Else, We are still pre-finalized.
+      // The existing directory should be left for backwards compatibility.
+      return VersionedDatanodeFeatures.ScmHA.
+          upgradeVolumeIfNeeded(volume, clusterId);
+    } else {
+      if (!clusterDir.exists()) {
+        logger.error("Volume {} is in an inconsistent state. {} files found " +
+            "but cluster ID directory {} does not exist.", volumeRootPath,
+            rootFiles.length, clusterDir);
+        return false;
+      }
+      return true;
+    }
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java
new file mode 100644
index 0000000..bd593d3
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolume.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_NAME;
+
+/**
+ * DbVolume represents a volume in datanode holding db instances
+ * for multiple HddsVolumes. One HddsVolume will have one subdirectory
+ * for its db instance under a DbVolume.
+ *
+ * For example:
+ *   Say we have an SSD device mounted at /ssd1, then the DbVolume
+ *   root directory is /ssd1/db, and we have a subdirectory
+ *   for db instance like
+ *   /ssd1/db/<clusterID>/<storageID>/container.db.
+ */
+public class DbVolume extends StorageVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DbVolume.class);
+
+  public static final String DB_VOLUME_DIR = "db";
+
+  /**
+   * Records all HddsVolumes that put its db instance under this DbVolume.
+   * Map: HddsVolume.StorageID -> DBStorePath
+   */
+  private final Map<String, String> hddsDbStorePathMap;
+
+  protected DbVolume(Builder b) throws IOException {
+    super(b);
+
+    this.hddsDbStorePathMap = new HashMap<>();
+    if (!b.getFailedVolume()) {
+      LOG.info("Creating DbVolume: {} of storage type : {} capacity : {}",
+          getStorageDir(), b.getStorageType(), getVolumeInfo().getCapacity());
+      initialize();
+    }
+  }
+
+  @Override
+  protected void initialize() throws IOException {
+    super.initialize();
+    scanForDbStorePaths();
+  }
+
+  @Override
+  public void failVolume() {
+    super.failVolume();
+    closeAllDbStore();
+  }
+
+  @Override
+  public void shutdown() {
+    super.shutdown();
+    closeAllDbStore();
+  }
+
+  public void addHddsDbStorePath(String id, String dbPath) {
+    hddsDbStorePathMap.put(id, dbPath);
+  }
+
+  public Set<String> getHddsVolumeIDs() {
+    return hddsDbStorePathMap.keySet();
+  }
+
+  /**
+   * Builder class for DbVolume.
+   */
+  public static class Builder extends StorageVolume.Builder<Builder> {
+
+    public Builder(String volumeRootStr) {
+      super(volumeRootStr, DB_VOLUME_DIR);
+    }
+
+    @Override
+    public Builder getThis() {
+      return this;
+    }
+
+    public DbVolume build() throws IOException {
+      return new DbVolume(this);
+    }
+  }
+
+  private void scanForDbStorePaths() throws IOException {
+    // Not formatted yet
+    if (!getStorageState().equals(VolumeState.NORMAL)) {
+      return;
+    }
+
+    // scan subdirectories for db instances mapped to HddsVolumes
+    File clusterIdDir = new File(getStorageDir(), getClusterID());
+    // Working dir not prepared yet
+    if (!clusterIdDir.exists()) {
+      return;
+    }
+
+    File[] subdirs = clusterIdDir.listFiles(File::isDirectory);
+    if (subdirs == null) {
+      throw new IOException("Failed to do listFiles for " +
+          clusterIdDir.getAbsolutePath());
+    }
+    hddsDbStorePathMap.clear();
+
+    for (File subdir : subdirs) {
+      String storageID = subdir.getName();
+      File storageIdDir = new File(clusterIdDir, subdir.getName());
+      hddsDbStorePathMap.put(storageID, new File(storageIdDir,
+          CONTAINER_DB_NAME).getAbsolutePath());
+    }
+  }
+
+  private void closeAllDbStore() {
+    // Here we check clusterID directly, because the state
+    // may not be NORMAL, it could be FAILED.
+    if (getClusterID() == null) {
+      return;
+    }
+
+    File clusterIdDir = new File(getStorageDir(), getClusterID());
+    if (clusterIdDir.exists()) {
+      for (String containerDBPath : hddsDbStorePathMap.values()) {
+        DatanodeStoreCache.getInstance().removeDB(containerDBPath);
+      }
+    }
+  }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolumeFactory.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolumeFactory.java
new file mode 100644
index 0000000..9aa4cef
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/DbVolumeFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.fs.SpaceUsageCheckFactory;
+
+import java.io.IOException;
+
+/**
+ * A factory class for DbVolume.
+ */
+public class DbVolumeFactory extends StorageVolumeFactory {
+
+  public DbVolumeFactory(ConfigurationSource conf,
+      SpaceUsageCheckFactory usageCheckFactory, MutableVolumeSet volumeSet,
+      String datanodeUuid, String clusterID) {
+    super(conf, usageCheckFactory, volumeSet, datanodeUuid, clusterID);
+  }
+
+  @Override
+  StorageVolume createVolume(String locationString, StorageType storageType)
+      throws IOException {
+    DbVolume.Builder volumeBuilder = new DbVolume.Builder(locationString)
+        .conf(getConf())
+        .datanodeUuid(getDatanodeUuid())
+        .clusterID(getClusterID())
+        .usageCheckFactory(getUsageCheckFactory())
+        .storageType(storageType)
+        .volumeSet(getVolumeSet());
+    DbVolume volume = volumeBuilder.build();
+
+    checkAndSetClusterID(volume.getClusterID());
+
+    return volume;
+  }
+
+  @Override
+  StorageVolume createFailedVolume(String locationString) throws IOException {
+    DbVolume.Builder volumeBuilder =
+        new DbVolume.Builder(locationString)
+            .failedVolume(true);
+    return volumeBuilder.build();
+  }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
index e817202..5bec415 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
@@ -18,25 +18,28 @@
 
 package org.apache.hadoop.ozone.container.common.volume;
 
-import static org.apache.hadoop.ozone.container.common.HDDSVolumeLayoutVersion.getLatestVersion;
-
 import java.io.File;
 import java.io.IOException;
-import java.util.Properties;
-import java.util.UUID;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
 import org.apache.hadoop.hdds.annotation.InterfaceStability;
-import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
-import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
-import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
-import org.apache.hadoop.util.Time;
 
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures.SchemaV3;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_NAME;
+import static org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil.initPerDiskDBStore;
+
 /**
  * HddsVolume represents volume in a datanode. {@link MutableVolumeSet}
  * maintains a list of HddsVolumes, one for each volume in the Datanode.
@@ -65,27 +68,26 @@
 
   public static final String HDDS_VOLUME_DIR = "hdds";
 
-  private VolumeState state;
   private final VolumeIOStats volumeIOStats;
   private final VolumeInfoMetrics volumeInfoMetrics;
 
-  // VERSION file properties
-  private String storageID;       // id of the file system
-  private String clusterID;       // id of the cluster
-  private String datanodeUuid;    // id of the DataNode
-  private long cTime;             // creation time of the file system state
-  private int layoutVersion;      // layout version of the storage data
   private final AtomicLong committedBytes; // till Open containers become full
 
   // Mentions the type of volume
   private final VolumeType type = VolumeType.DATA_VOLUME;
+  // The dedicated DbVolume that the db instance of this HddsVolume resides.
+  // This is optional, if null then the db instance resides on this HddsVolume.
+  private DbVolume dbVolume;
+  // The subdirectory with storageID as its name, used to build the
+  // container db path. This is initialized only once together with dbVolume,
+  // and stored as a member to prevent spawning lots of File objects.
+  private File dbParentDir;
+  private AtomicBoolean dbLoaded = new AtomicBoolean(false);
 
   /**
    * Builder for HddsVolume.
    */
   public static class Builder extends StorageVolume.Builder<Builder> {
-    private String datanodeUuid;
-    private String clusterID;
 
     public Builder(String volumeRootStr) {
       super(volumeRootStr, HDDS_VOLUME_DIR);
@@ -96,16 +98,6 @@
       return this;
     }
 
-    public Builder datanodeUuid(String datanodeUUID) {
-      this.datanodeUuid = datanodeUUID;
-      return this;
-    }
-
-    public Builder clusterID(String cid) {
-      this.clusterID = cid;
-      return this;
-    }
-
     public HddsVolume build() throws IOException {
       return new HddsVolume(this);
     }
@@ -115,9 +107,7 @@
     super(b);
 
     if (!b.getFailedVolume()) {
-      this.state = VolumeState.NOT_INITIALIZED;
-      this.clusterID = b.clusterID;
-      this.datanodeUuid = b.datanodeUuid;
+      this.setState(VolumeState.NOT_INITIALIZED);
       this.volumeIOStats = new VolumeIOStats(b.getVolumeRootStr());
       this.volumeInfoMetrics =
           new VolumeInfoMetrics(b.getVolumeRootStr(), this);
@@ -130,201 +120,34 @@
     } else {
       // Builder is called with failedVolume set, so create a failed volume
       // HddsVolume Object.
+      this.setState(VolumeState.FAILED);
       volumeIOStats = null;
       volumeInfoMetrics = null;
-      storageID = UUID.randomUUID().toString();
-      state = VolumeState.FAILED;
       committedBytes = null;
     }
 
   }
 
-  /**
-   * Initializes the volume.
-   * Creates the Version file if not present,
-   * otherwise returns with IOException.
-   *
-   * @throws IOException
-   */
-  private void initialize() throws IOException {
-    VolumeState intialVolumeState = analyzeVolumeState();
-    switch (intialVolumeState) {
-    case NON_EXISTENT:
-      // Root directory does not exist. Create it.
-      if (!getStorageDir().mkdirs()) {
-        throw new IOException("Cannot create directory " + getStorageDir());
-      }
-      setState(VolumeState.NOT_FORMATTED);
-      createVersionFile();
-      break;
-    case NOT_FORMATTED:
-      // Version File does not exist. Create it.
-      createVersionFile();
-      break;
-    case NOT_INITIALIZED:
-      // Version File exists. Verify its correctness and update property fields.
-      readVersionFile();
-      setState(VolumeState.NORMAL);
-      break;
-    case INCONSISTENT:
-      // Volume Root is in an inconsistent state. Skip loading this volume.
-      throw new IOException("Volume is in an " + VolumeState.INCONSISTENT +
-          " state. Skipped loading volume: " + getStorageDir().getPath());
-    default:
-      throw new IOException("Unrecognized initial state : " +
-          intialVolumeState + "of volume : " + getStorageDir());
+  @Override
+  public void createWorkingDir(String workingDirName,
+      MutableVolumeSet dbVolumeSet) throws IOException {
+    super.createWorkingDir(workingDirName, dbVolumeSet);
+
+    // Create DB store for a newly formatted volume
+    if (VersionedDatanodeFeatures.isFinalized(
+        HDDSLayoutFeature.DATANODE_SCHEMA_V3)) {
+      createDbStore(dbVolumeSet);
     }
   }
 
-  private VolumeState analyzeVolumeState() {
-    if (!getStorageDir().exists()) {
-      // Volume Root does not exist.
-      return VolumeState.NON_EXISTENT;
-    }
-    if (!getStorageDir().isDirectory()) {
-      // Volume Root exists but is not a directory.
-      LOG.warn("Volume {} exists but is not a directory,"
-              + " current volume state: {}.",
-          getStorageDir().getPath(), VolumeState.INCONSISTENT);
-      return VolumeState.INCONSISTENT;
-    }
-    File[] files = getStorageDir().listFiles();
-    if (files == null || files.length == 0) {
-      // Volume Root exists and is empty.
-      return VolumeState.NOT_FORMATTED;
-    }
-    if (!getVersionFile().exists()) {
-      // Volume Root is non empty but VERSION file does not exist.
-      LOG.warn("VERSION file does not exist in volume {},"
-              + " current volume state: {}.",
-          getStorageDir().getPath(), VolumeState.INCONSISTENT);
-      return VolumeState.INCONSISTENT;
-    }
-    // Volume Root and VERSION file exist.
-    return VolumeState.NOT_INITIALIZED;
-  }
-
-  public void format(String cid) throws IOException {
-    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
-        "formatting Volume");
-    this.clusterID = cid;
-    initialize();
-  }
-
-  /**
-   * Create Version File and write property fields into it.
-   *
-   * @throws IOException
-   */
-  private void createVersionFile() throws IOException {
-    this.storageID = HddsVolumeUtil.generateUuid();
-    this.cTime = Time.now();
-    this.layoutVersion = getLatestVersion().getVersion();
-
-    if (this.clusterID == null || datanodeUuid == null) {
-      // HddsDatanodeService does not have the cluster information yet. Wait
-      // for registration with SCM.
-      LOG.debug("ClusterID not available. Cannot format the volume {}",
-          getStorageDir().getPath());
-      setState(VolumeState.NOT_FORMATTED);
-    } else {
-      // Write the version file to disk.
-      writeVersionFile();
-      setState(VolumeState.NORMAL);
-    }
-  }
-
-  private void writeVersionFile() throws IOException {
-    Preconditions.checkNotNull(this.storageID,
-        "StorageID cannot be null in Version File");
-    Preconditions.checkNotNull(this.clusterID,
-        "ClusterID cannot be null in Version File");
-    Preconditions.checkNotNull(this.datanodeUuid,
-        "DatanodeUUID cannot be null in Version File");
-    Preconditions.checkArgument(this.cTime > 0,
-        "Creation Time should be positive");
-    Preconditions.checkArgument(this.layoutVersion ==
-            getLatestVersion().getVersion(),
-        "Version File should have the latest LayOutVersion");
-
-    File versionFile = getVersionFile();
-    LOG.debug("Writing Version file to disk, {}", versionFile);
-
-    DatanodeVersionFile dnVersionFile = new DatanodeVersionFile(this.storageID,
-        this.clusterID, this.datanodeUuid, this.cTime, this.layoutVersion);
-    dnVersionFile.createVersionFile(versionFile);
-  }
-
-  /**
-   * Read Version File and update property fields.
-   * Get common storage fields.
-   * Should be overloaded if additional fields need to be read.
-   *
-   * @throws IOException on error
-   */
-  private void readVersionFile() throws IOException {
-    File versionFile = getVersionFile();
-    Properties props = DatanodeVersionFile.readFrom(versionFile);
-    if (props.isEmpty()) {
-      throw new InconsistentStorageStateException(
-          "Version file " + versionFile + " is missing");
-    }
-
-    LOG.debug("Reading Version file from disk, {}", versionFile);
-    this.storageID = HddsVolumeUtil.getStorageID(props, versionFile);
-    this.clusterID = HddsVolumeUtil.getClusterID(props, versionFile,
-        this.clusterID);
-    this.datanodeUuid = HddsVolumeUtil.getDatanodeUUID(props, versionFile,
-        this.datanodeUuid);
-    this.cTime = HddsVolumeUtil.getCreationTime(props, versionFile);
-    this.layoutVersion = HddsVolumeUtil.getLayOutVersion(props, versionFile);
-  }
-
-  private File getVersionFile() {
-    return HddsVolumeUtil.getVersionFile(super.getStorageDir());
-  }
-
   public File getHddsRootDir() {
     return super.getStorageDir();
   }
 
-  @Override
-  public String getStorageID() {
-    return storageID;
-  }
-
-  public String getClusterID() {
-    return clusterID;
-  }
-
-  public String getDatanodeUuid() {
-    return datanodeUuid;
-  }
-
-  public long getCTime() {
-    return cTime;
-  }
-
-  public int getLayoutVersion() {
-    return layoutVersion;
-  }
-
   public VolumeType getType() {
     return type;
   }
 
-  public VolumeState getStorageState() {
-    return state;
-  }
-
-  public void setState(VolumeState state) {
-    this.state = state;
-  }
-
-  public boolean isFailed() {
-    return (state == VolumeState.FAILED);
-  }
-
   public VolumeIOStats getVolumeIOStats() {
     return volumeIOStats;
   }
@@ -335,7 +158,6 @@
 
   @Override
   public void failVolume() {
-    setState(VolumeState.FAILED);
     super.failVolume();
     if (volumeIOStats != null) {
       volumeIOStats.unregister();
@@ -343,11 +165,11 @@
     if (volumeInfoMetrics != null) {
       volumeInfoMetrics.unregister();
     }
+    closeDbStore();
   }
 
   @Override
   public void shutdown() {
-    this.state = VolumeState.NON_EXISTENT;
     super.shutdown();
     if (volumeIOStats != null) {
       volumeIOStats.unregister();
@@ -355,27 +177,7 @@
     if (volumeInfoMetrics != null) {
       volumeInfoMetrics.unregister();
     }
-  }
-
-  /**
-   * VolumeState represents the different states a HddsVolume can be in.
-   * NORMAL          =&gt; Volume can be used for storage
-   * FAILED          =&gt; Volume has failed due and can no longer be used for
-   * storing containers.
-   * NON_EXISTENT    =&gt; Volume Root dir does not exist
-   * INCONSISTENT    =&gt; Volume Root dir is not empty but VERSION file is
-   * missing or Volume Root dir is not a directory
-   * NOT_FORMATTED   =&gt; Volume Root exists but not formatted(no VERSION file)
-   * NOT_INITIALIZED =&gt; VERSION file exists but has not been verified for
-   * correctness.
-   */
-  public enum VolumeState {
-    NORMAL,
-    FAILED,
-    NON_EXISTENT,
-    INCONSISTENT,
-    NOT_FORMATTED,
-    NOT_INITIALIZED
+    closeDbStore();
   }
 
   /**
@@ -396,4 +198,146 @@
   public long getCommittedBytes() {
     return committedBytes.get();
   }
-}
\ No newline at end of file
+
+  public void setDbVolume(DbVolume dbVolume) {
+    this.dbVolume = dbVolume;
+  }
+
+  public DbVolume getDbVolume() {
+    return this.dbVolume;
+  }
+
+  public File getDbParentDir() {
+    return this.dbParentDir;
+  }
+
+  public boolean isDbLoaded() {
+    return dbLoaded.get();
+  }
+
+  public void loadDbStore() throws IOException {
+    // DN startup for the first time, not registered yet,
+    // so the DbVolume is not formatted.
+    if (!getStorageState().equals(VolumeState.NORMAL)) {
+      return;
+    }
+
+    // DB is already loaded
+    if (dbLoaded.get()) {
+      LOG.warn("Schema V3 db is already loaded from {} for volume {}",
+          getDbParentDir(), getStorageID());
+      return;
+    }
+
+    File clusterIdDir = new File(dbVolume == null ?
+        getStorageDir() : dbVolume.getStorageDir(),
+        getClusterID());
+    if (!clusterIdDir.exists()) {
+      throw new IOException("Working dir " + clusterIdDir.getAbsolutePath() +
+          " not created for HddsVolume: " + getStorageDir().getAbsolutePath());
+    }
+
+    File storageIdDir = new File(clusterIdDir, getStorageID());
+    if (!storageIdDir.exists()) {
+      throw new IOException("Db parent dir " + storageIdDir.getAbsolutePath() +
+          " not found for HddsVolume: " + getStorageDir().getAbsolutePath());
+    }
+
+    File containerDBFile = new File(storageIdDir, CONTAINER_DB_NAME);
+    if (!containerDBFile.exists()) {
+      throw new IOException("Db dir " + storageIdDir.getAbsolutePath() +
+          " not found for HddsVolume: " + getStorageDir().getAbsolutePath());
+    }
+
+    String containerDBPath = containerDBFile.getAbsolutePath();
+    try {
+      initPerDiskDBStore(containerDBPath, getConf());
+    } catch (IOException e) {
+      throw new IOException("Can't init db instance under path "
+          + containerDBPath + " for volume " + getStorageID(), e);
+    }
+
+    dbParentDir = storageIdDir;
+    dbLoaded.set(true);
+    LOG.info("SchemaV3 db is loaded at {} for volume {}", containerDBPath,
+        getStorageID());
+  }
+
+  /**
+   * Pick a DbVolume for HddsVolume and init db instance.
+   * Use the HddsVolume directly if no DbVolume found.
+   * @param dbVolumeSet
+   */
+  public void createDbStore(MutableVolumeSet dbVolumeSet) throws IOException {
+    DbVolume chosenDbVolume = null;
+    File clusterIdDir;
+    String workingDir = getWorkingDir() == null ? getClusterID() :
+        getWorkingDir();
+
+    if (dbVolumeSet == null || dbVolumeSet.getVolumesList().isEmpty()) {
+      // No extra db volumes specified, just create db under the HddsVolume.
+      clusterIdDir = new File(getStorageDir(), workingDir);
+    } else {
+      // Randomly choose a DbVolume for simplicity.
+      List<DbVolume> dbVolumeList = StorageVolumeUtil.getDbVolumesList(
+          dbVolumeSet.getVolumesList());
+      chosenDbVolume = dbVolumeList.get(
+          ThreadLocalRandom.current().nextInt(dbVolumeList.size()));
+      clusterIdDir = new File(chosenDbVolume.getStorageDir(), workingDir);
+    }
+
+    if (!clusterIdDir.exists()) {
+      throw new IOException("The working dir "
+          + clusterIdDir.getAbsolutePath() + " is missing for volume "
+          + getStorageID());
+    }
+
+    // Init subdir with the storageID of HddsVolume.
+    File storageIdDir = new File(clusterIdDir, getStorageID());
+    if (!storageIdDir.mkdirs() && !storageIdDir.exists()) {
+      throw new IOException("Can't make subdir under "
+          + clusterIdDir.getAbsolutePath() + " for volume "
+          + getStorageID());
+    }
+
+    // Create the db instance for HddsVolume under the subdir above.
+    String containerDBPath = new File(storageIdDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    try {
+      HddsVolumeUtil.initPerDiskDBStore(containerDBPath, getConf());
+      dbLoaded.set(true);
+      LOG.info("SchemaV3 db is created and loaded at {} for volume {}",
+          containerDBPath, getStorageID());
+    } catch (IOException e) {
+      String errMsg = "Can't create db instance under path "
+          + containerDBPath + " for volume " + getStorageID();
+      LOG.error(errMsg, e);
+      throw new IOException(errMsg);
+    }
+
+    // Set the dbVolume and dbParentDir of the HddsVolume for db path lookup.
+    dbVolume = chosenDbVolume;
+    dbParentDir = storageIdDir;
+    if (chosenDbVolume != null) {
+      chosenDbVolume.addHddsDbStorePath(getStorageID(), containerDBPath);
+    }
+
+    // If SchemaV3 is disabled, close the DB instance
+    if (!SchemaV3.isFinalizedAndEnabled(getConf())) {
+      closeDbStore();
+    }
+  }
+
+  private void closeDbStore() {
+    if (!dbLoaded.get()) {
+      return;
+    }
+
+    String containerDBPath = new File(dbParentDir, CONTAINER_DB_NAME)
+        .getAbsolutePath();
+    DatanodeStoreCache.getInstance().removeDB(containerDBPath);
+    dbLoaded.set(false);
+    LOG.info("SchemaV3 db is stopped at {} for volume {}", containerDBPath,
+        getStorageID());
+  }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolumeFactory.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolumeFactory.java
index 3b7b108..afb3016 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolumeFactory.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolumeFactory.java
@@ -21,7 +21,6 @@
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.fs.SpaceUsageCheckFactory;
-import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
 
 import java.io.IOException;
 
@@ -30,15 +29,10 @@
  */
 public class HddsVolumeFactory extends StorageVolumeFactory {
 
-  private String datanodeUuid;
-  private String clusterID;
-
   public HddsVolumeFactory(ConfigurationSource conf,
       SpaceUsageCheckFactory usageCheckFactory, MutableVolumeSet volumeSet,
       String datanodeUuid, String clusterID) {
-    super(conf, usageCheckFactory, volumeSet);
-    this.datanodeUuid = datanodeUuid;
-    this.clusterID = clusterID;
+    super(conf, usageCheckFactory, volumeSet, datanodeUuid, clusterID);
   }
 
   @Override
@@ -46,8 +40,8 @@
       StorageType storageType) throws IOException {
     HddsVolume.Builder volumeBuilder = new HddsVolume.Builder(locationString)
         .conf(getConf())
-        .datanodeUuid(datanodeUuid)
-        .clusterID(clusterID)
+        .datanodeUuid(getDatanodeUuid())
+        .clusterID(getClusterID())
         .usageCheckFactory(getUsageCheckFactory())
         .storageType(storageType)
         .volumeSet(getVolumeSet());
@@ -65,29 +59,4 @@
         .failedVolume(true);
     return volumeBuilder.build();
   }
-
-  /**
-   * If Version file exists and the {@link #clusterID} is not set yet,
-   * assign it the value from Version file. Otherwise, check that the given
-   * id matches with the id from version file.
-   * @param idFromVersionFile value of the property from Version file
-   * @throws InconsistentStorageStateException
-   */
-  private void checkAndSetClusterID(String idFromVersionFile)
-      throws InconsistentStorageStateException {
-    // If the clusterID is null (not set), assign it the value
-    // from version file.
-    if (this.clusterID == null) {
-      this.clusterID = idFromVersionFile;
-      return;
-    }
-
-    // If the clusterID is already set, it should match with the value from the
-    // version file.
-    if (!idFromVersionFile.equals(this.clusterID)) {
-      throw new InconsistentStorageStateException(
-          "Mismatched ClusterIDs. VolumeSet has: " + this.clusterID +
-              ", and version file has: " + idFromVersionFile);
-    }
-  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MetadataVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MetadataVolume.java
index bdedeff..c5b399b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MetadataVolume.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MetadataVolume.java
@@ -55,4 +55,9 @@
       return new MetadataVolume(this);
     }
   }
+
+  @Override
+  public String getStorageID() {
+    return "";
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MetadataVolumeFactory.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MetadataVolumeFactory.java
index b83cb38..cffe5b7 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MetadataVolumeFactory.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MetadataVolumeFactory.java
@@ -31,7 +31,7 @@
 
   public MetadataVolumeFactory(ConfigurationSource conf,
       SpaceUsageCheckFactory usageCheckFactory, MutableVolumeSet volumeSet) {
-    super(conf, usageCheckFactory, volumeSet);
+    super(conf, usageCheckFactory, volumeSet, null, null);
   }
 
   @Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
index 98e1629..2257168 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
@@ -121,6 +121,10 @@
       this.volumeFactory = new MetadataVolumeFactory(conf, usageCheckFactory,
           this);
       maxVolumeFailuresTolerated = dnConf.getFailedMetadataVolumesTolerated();
+    } else if (volumeType == StorageVolume.VolumeType.DB_VOLUME) {
+      this.volumeFactory = new DbVolumeFactory(conf, usageCheckFactory,
+          this, datanodeUuid, clusterID);
+      maxVolumeFailuresTolerated = dnConf.getFailedDbVolumesTolerated();
     } else {
       this.volumeFactory = new HddsVolumeFactory(conf, usageCheckFactory,
           this, datanodeUuid, clusterID);
@@ -150,6 +154,8 @@
     Collection<String> rawLocations;
     if (volumeType == StorageVolume.VolumeType.META_VOLUME) {
       rawLocations = HddsServerUtil.getOzoneDatanodeRatisDirectory(conf);
+    } else if (volumeType == StorageVolume.VolumeType.DB_VOLUME) {
+      rawLocations = HddsServerUtil.getDatanodeDbDirs(conf);
     } else {
       rawLocations = HddsServerUtil.getDatanodeStorageDirs(conf);
     }
@@ -449,6 +455,11 @@
   }
 
   @VisibleForTesting
+  public void setVolumeMap(Map<String, StorageVolume> map) {
+    this.volumeMap = map;
+  }
+
+  @VisibleForTesting
   public Map<StorageType, List<StorageVolume>> getVolumeStateMap() {
     return ImmutableMap.copyOf(volumeStateMap);
   }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java
index 715cb84..18892b6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java
@@ -18,41 +18,99 @@
 
 package org.apache.hadoop.ozone.container.common.volume;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.fs.SpaceUsageCheckFactory;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
 import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
 import java.io.File;
 import java.io.IOException;
 import java.util.Objects;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.apache.hadoop.ozone.container.common.HDDSVolumeLayoutVersion.getLatestVersion;
 
 /**
  * StorageVolume represents a generic Volume in datanode, could be
  * 1. HddsVolume for container storage.
  * 2. MetadataVolume for metadata(ratis) storage.
+ *    This is a special type of volume, because it is managed
+ *    by ratis itself, so we don't format or initialize it in Ozone.
+ * 3. DbVolume for db instance storage.
+ *
+ * Each hdds volume has its own VERSION file. The hdds volume will have one
+ * clusterUuid directory for each SCM it is a part of.
+ *
+ * During DN startup, if the VERSION file exists, we verify that the
+ * clusterID in the version file matches the clusterID from SCM.
  */
 public abstract class StorageVolume
     implements Checkable<Boolean, VolumeCheckResult> {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StorageVolume.class);
+
   /**
    * Type for StorageVolume.
    */
   public enum VolumeType {
     DATA_VOLUME,
-    META_VOLUME
+    META_VOLUME,
+    DB_VOLUME,
   }
 
+  /**
+   * VolumeState represents the different states a StorageVolume can be in.
+   * NORMAL          =&gt; Volume can be used for storage
+   * FAILED          =&gt; Volume has failed due and can no longer be used for
+   *                    storing containers.
+   * NON_EXISTENT    =&gt; Volume Root dir does not exist
+   * INCONSISTENT    =&gt; Volume Root dir is not empty but VERSION file is
+   *                    missing or Volume Root dir is not a directory
+   * NOT_FORMATTED   =&gt; Volume Root exists but not formatted(no VERSION file)
+   * NOT_INITIALIZED =&gt; VERSION file exists but has not been verified for
+   *                    correctness.
+   */
+  public enum VolumeState {
+    NORMAL,
+    FAILED,
+    NON_EXISTENT,
+    INCONSISTENT,
+    NOT_FORMATTED,
+    NOT_INITIALIZED
+  }
+
+  private VolumeState state;
+
+  // VERSION file properties
+  private String storageID;       // id of the file system
+  private String clusterID;       // id of the cluster
+  private String datanodeUuid;    // id of the DataNode
+  private long cTime;             // creation time of the file system state
+  private int layoutVersion;      // layout version of the storage data
+
+  private ConfigurationSource conf;
+
   private final File storageDir;
 
   private final VolumeInfo volumeInfo;
 
   private final VolumeSet volumeSet;
 
+  private String workingDir;
+
   protected StorageVolume(Builder<?> b) throws IOException {
     if (!b.failedVolume) {
       StorageLocation location = StorageLocation.parse(b.volumeRootStr);
@@ -62,13 +120,179 @@
           .usageCheckFactory(b.usageCheckFactory)
           .build();
       this.volumeSet = b.volumeSet;
+      this.state = VolumeState.NOT_INITIALIZED;
+      this.clusterID = b.clusterID;
+      this.datanodeUuid = b.datanodeUuid;
+      this.conf = b.conf;
     } else {
       storageDir = new File(b.volumeRootStr);
       this.volumeInfo = null;
       this.volumeSet = null;
+      this.storageID = UUID.randomUUID().toString();
+      this.state = VolumeState.FAILED;
     }
   }
 
+  public void format(String cid) throws IOException {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting Volume");
+    this.clusterID = cid;
+    initialize();
+  }
+
+  /**
+   * Initializes the volume.
+   * Creates the Version file if not present,
+   * otherwise returns with IOException.
+   * @throws IOException
+   */
+  protected void initialize() throws IOException {
+    VolumeState intialVolumeState = analyzeVolumeState();
+    switch (intialVolumeState) {
+    case NON_EXISTENT:
+      // Root directory does not exist. Create it.
+      if (!getStorageDir().mkdirs()) {
+        throw new IOException("Cannot create directory " + getStorageDir());
+      }
+      setState(VolumeState.NOT_FORMATTED);
+      createVersionFile();
+      break;
+    case NOT_FORMATTED:
+      // Version File does not exist. Create it.
+      createVersionFile();
+      break;
+    case NOT_INITIALIZED:
+      // Version File exists.
+      // Verify its correctness and update property fields.
+      readVersionFile();
+      setState(VolumeState.NORMAL);
+      break;
+    case INCONSISTENT:
+      // Volume Root is in an inconsistent state. Skip loading this volume.
+      throw new IOException("Volume is in an " + VolumeState.INCONSISTENT +
+          " state. Skipped loading volume: " + getStorageDir().getPath());
+    default:
+      throw new IOException("Unrecognized initial state : " +
+          intialVolumeState + "of volume : " + getStorageDir());
+    }
+  }
+
+  /**
+   * Create working directory for cluster io loads.
+   * @param workingDirName scmID or clusterID according to SCM HA config
+   * @param dbVolumeSet optional dbVolumes
+   * @throws IOException
+   */
+  public void createWorkingDir(String workingDirName,
+      MutableVolumeSet dbVolumeSet) throws IOException {
+    File idDir = new File(getStorageDir(), workingDirName);
+    if (!idDir.mkdir()) {
+      throw new IOException("Unable to create ID directory " + idDir +
+          " for datanode.");
+    }
+    this.workingDir = workingDirName;
+  }
+
+  private VolumeState analyzeVolumeState() {
+    if (!getStorageDir().exists()) {
+      // Volume Root does not exist.
+      return VolumeState.NON_EXISTENT;
+    }
+    if (!getStorageDir().isDirectory()) {
+      // Volume Root exists but is not a directory.
+      LOG.warn("Volume {} exists but is not a directory,"
+              + " current volume state: {}.",
+          getStorageDir().getPath(), VolumeState.INCONSISTENT);
+      return VolumeState.INCONSISTENT;
+    }
+    File[] files = getStorageDir().listFiles();
+    if (files == null || files.length == 0) {
+      // Volume Root exists and is empty.
+      return VolumeState.NOT_FORMATTED;
+    }
+    if (!getVersionFile().exists()) {
+      // Volume Root is non empty but VERSION file does not exist.
+      LOG.warn("VERSION file does not exist in volume {},"
+              + " current volume state: {}.",
+          getStorageDir().getPath(), VolumeState.INCONSISTENT);
+      return VolumeState.INCONSISTENT;
+    }
+    // Volume Root and VERSION file exist.
+    return VolumeState.NOT_INITIALIZED;
+  }
+
+  /**
+   * Create Version File and write property fields into it.
+   * @throws IOException
+   */
+  private void createVersionFile() throws IOException {
+    this.storageID = StorageVolumeUtil.generateUuid();
+    this.cTime = Time.now();
+    this.layoutVersion = getLatestVersion().getVersion();
+
+    if (this.clusterID == null || datanodeUuid == null) {
+      // HddsDatanodeService does not have the cluster information yet. Wait
+      // for registration with SCM.
+      LOG.debug("ClusterID not available. Cannot format the volume {}",
+          getStorageDir().getPath());
+      setState(VolumeState.NOT_FORMATTED);
+    } else {
+      // Write the version file to disk.
+      writeVersionFile();
+      setState(VolumeState.NORMAL);
+    }
+  }
+
+  private void writeVersionFile() throws IOException {
+    Preconditions.checkNotNull(this.storageID,
+        "StorageID cannot be null in Version File");
+    Preconditions.checkNotNull(this.clusterID,
+        "ClusterID cannot be null in Version File");
+    Preconditions.checkNotNull(this.datanodeUuid,
+        "DatanodeUUID cannot be null in Version File");
+    Preconditions.checkArgument(this.cTime > 0,
+        "Creation Time should be positive");
+    Preconditions.checkArgument(this.layoutVersion ==
+            getLatestVersion().getVersion(),
+        "Version File should have the latest LayOutVersion");
+
+    File versionFile = getVersionFile();
+    LOG.debug("Writing Version file to disk, {}", versionFile);
+
+    DatanodeVersionFile dnVersionFile = new DatanodeVersionFile(this.storageID,
+        this.clusterID, this.datanodeUuid, this.cTime, this.layoutVersion);
+    dnVersionFile.createVersionFile(versionFile);
+  }
+
+  /**
+   * Read Version File and update property fields.
+   * Get common storage fields.
+   * Should be overloaded if additional fields need to be read.
+   *
+   * @throws IOException on error
+   */
+  private void readVersionFile() throws IOException {
+    File versionFile = getVersionFile();
+    Properties props = DatanodeVersionFile.readFrom(versionFile);
+    if (props.isEmpty()) {
+      throw new InconsistentStorageStateException(
+          "Version file " + versionFile + " is missing");
+    }
+
+    LOG.debug("Reading Version file from disk, {}", versionFile);
+    this.storageID = StorageVolumeUtil.getStorageID(props, versionFile);
+    this.clusterID = StorageVolumeUtil.getClusterID(props, versionFile,
+        this.clusterID);
+    this.datanodeUuid = StorageVolumeUtil.getDatanodeUUID(props, versionFile,
+        this.datanodeUuid);
+    this.cTime = StorageVolumeUtil.getCreationTime(props, versionFile);
+    this.layoutVersion = StorageVolumeUtil.getLayOutVersion(props, versionFile);
+  }
+
+  private File getVersionFile() {
+    return StorageVolumeUtil.getVersionFile(getStorageDir());
+  }
+
   /**
    * Builder class for StorageVolume.
    * @param <T> subclass Builder
@@ -81,6 +305,8 @@
     private SpaceUsageCheckFactory usageCheckFactory;
     private VolumeSet volumeSet;
     private boolean failedVolume = false;
+    private String datanodeUuid;
+    private String clusterID;
 
     public Builder(String volumeRootStr, String storageDirStr) {
       this.volumeRootStr = volumeRootStr;
@@ -117,6 +343,16 @@
       return this.getThis();
     }
 
+    public T datanodeUuid(String datanodeUUID) {
+      this.datanodeUuid = datanodeUUID;
+      return this.getThis();
+    }
+
+    public T clusterID(String cid) {
+      this.clusterID = cid;
+      return this.getThis();
+    }
+
     public abstract StorageVolume build() throws IOException;
 
     public String getVolumeRootStr() {
@@ -148,6 +384,10 @@
     return this.storageDir;
   }
 
+  public String getWorkingDir() {
+    return this.workingDir;
+  }
+
   public void refreshVolumeInfo() {
     volumeInfo.refreshNow();
   }
@@ -168,16 +408,50 @@
   }
 
   public String getStorageID() {
-    return "";
+    return storageID;
+  }
+
+  public String getClusterID() {
+    return clusterID;
+  }
+
+  public String getDatanodeUuid() {
+    return datanodeUuid;
+  }
+
+  public long getCTime() {
+    return cTime;
+  }
+
+  public int getLayoutVersion() {
+    return layoutVersion;
+  }
+
+  public VolumeState getStorageState() {
+    return state;
+  }
+
+  public void setState(VolumeState state) {
+    this.state = state;
+  }
+
+  public boolean isFailed() {
+    return (state == VolumeState.FAILED);
+  }
+
+  public ConfigurationSource getConf() {
+    return conf;
   }
 
   public void failVolume() {
+    setState(VolumeState.FAILED);
     if (volumeInfo != null) {
       volumeInfo.shutdownUsageThread();
     }
   }
 
   public void shutdown() {
+    setState(VolumeState.NON_EXISTENT);
     if (volumeInfo != null) {
       volumeInfo.shutdownUsageThread();
     }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolumeFactory.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolumeFactory.java
index 9273f35..7527eb8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolumeFactory.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolumeFactory.java
@@ -21,6 +21,7 @@
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.fs.SpaceUsageCheckFactory;
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
 
 import java.io.IOException;
 
@@ -32,12 +33,17 @@
   private ConfigurationSource conf;
   private SpaceUsageCheckFactory usageCheckFactory;
   private MutableVolumeSet volumeSet;
+  private String datanodeUuid;
+  private String clusterID;
 
   public StorageVolumeFactory(ConfigurationSource conf,
-      SpaceUsageCheckFactory usageCheckFactory, MutableVolumeSet volumeSet) {
+      SpaceUsageCheckFactory usageCheckFactory, MutableVolumeSet volumeSet,
+      String datanodeUuid, String clusterID) {
     this.conf = conf;
     this.usageCheckFactory = usageCheckFactory;
     this.volumeSet = volumeSet;
+    this.datanodeUuid = datanodeUuid;
+    this.clusterID = clusterID;
   }
 
   public ConfigurationSource getConf() {
@@ -52,6 +58,39 @@
     return this.volumeSet;
   }
 
+  public String getDatanodeUuid() {
+    return this.datanodeUuid;
+  }
+
+  public String getClusterID() {
+    return this.clusterID;
+  }
+
+  /**
+   * If Version file exists and the {@link #clusterID} is not set yet,
+   * assign it the value from Version file. Otherwise, check that the given
+   * id matches with the id from version file.
+   * @param idFromVersionFile value of the property from Version file
+   * @throws InconsistentStorageStateException
+   */
+  protected void checkAndSetClusterID(String idFromVersionFile)
+      throws InconsistentStorageStateException {
+    // If the clusterID is null (not set), assign it the value
+    // from version file.
+    if (this.clusterID == null) {
+      this.clusterID = idFromVersionFile;
+      return;
+    }
+
+    // If the clusterID is already set, it should match with the value from the
+    // version file.
+    if (!idFromVersionFile.equals(this.clusterID)) {
+      throw new InconsistentStorageStateException(
+          "Mismatched ClusterIDs. VolumeSet has: " + this.clusterID +
+              ", and version file has: " + idFromVersionFile);
+    }
+  }
+
   abstract StorageVolume createVolume(String locationString,
       StorageType storageType) throws IOException;
 
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index fdf44ed..742e101 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -47,8 +47,8 @@
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
@@ -88,6 +88,8 @@
   // Use a non-fair RW lock for better throughput, we may revisit this decision
   // if this causes fairness issues.
   private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  // Simple lock to synchronize container metadata dump operation.
+  private final Object dumpLock = new Object();
 
   private final KeyValueContainerData containerData;
   private ConfigurationSource config;
@@ -137,10 +139,18 @@
           StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()),
           maxSize);
       String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
+      // Set volume before getContainerDBFile(), because we may need the
+      // volume to deduce the db file.
+      containerData.setVolume(containerVolume);
 
       long containerID = containerData.getContainerID();
       String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(
               containerVolume, clusterId);
+      // Set schemaVersion before the dbFile since we have to
+      // choose the dbFile location based on schema version.
+      String schemaVersion = VersionedDatanodeFeatures.SchemaV3
+          .chooseSchemaVersion(config);
+      containerData.setSchemaVersion(schemaVersion);
 
       containerMetaDataPath = KeyValueContainerLocationUtil
           .getContainerMetaDataPath(hddsVolumeDir, idDir, containerID);
@@ -155,16 +165,13 @@
       //Create Metadata path chunks path and metadata db
       File dbFile = getContainerDBFile();
 
-      containerData.setSchemaVersion(
-          VersionedDatanodeFeatures.SchemaV2.chooseSchemaVersion());
-      KeyValueContainerUtil.createContainerMetaData(containerID,
+      KeyValueContainerUtil.createContainerMetaData(
               containerMetaDataPath, chunksPath, dbFile,
               containerData.getSchemaVersion(), config);
 
       //Set containerData for the KeyValueContainer.
       containerData.setChunksPath(chunksPath.getPath());
       containerData.setDbFile(dbFile);
-      containerData.setVolume(containerVolume);
 
       // Create .container file
       File containerFile = getContainerFile();
@@ -202,26 +209,24 @@
    *
    * @param clusterId
    * @param containerVolume
-   * @param hddsVolumeDir
    */
   public void populatePathFields(String clusterId,
-      HddsVolume containerVolume, String hddsVolumeDir) {
+      HddsVolume containerVolume) {
 
     long containerId = containerData.getContainerID();
+    String hddsVolumeDir = containerVolume.getHddsRootDir().getAbsolutePath();
 
     File containerMetaDataPath = KeyValueContainerLocationUtil
         .getContainerMetaDataPath(hddsVolumeDir, clusterId, containerId);
 
     File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(
         hddsVolumeDir, clusterId, containerId);
-    File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
-        containerMetaDataPath, containerId);
 
     //Set containerData for the KeyValueContainer.
     containerData.setMetadataPath(containerMetaDataPath.getPath());
     containerData.setChunksPath(chunksPath.getPath());
-    containerData.setDbFile(dbFile);
     containerData.setVolume(containerVolume);
+    containerData.setDbFile(getContainerDBFile());
   }
 
   /**
@@ -331,36 +336,12 @@
 
   @Override
   public void quasiClose() throws StorageContainerException {
-    // The DB must be synced during close operation
-    flushAndSyncDB();
-
-    writeLock();
-    try {
-      // Second sync should be a very light operation as sync has already
-      // been done outside the lock.
-      flushAndSyncDB();
-      updateContainerData(containerData::quasiCloseContainer);
-      clearPendingPutBlockCache();
-    } finally {
-      writeUnlock();
-    }
+    closeAndFlushIfNeeded(containerData::quasiCloseContainer);
   }
 
   @Override
   public void close() throws StorageContainerException {
-    // The DB must be synced during close operation
-    flushAndSyncDB();
-
-    writeLock();
-    try {
-      // Second sync should be a very light operation as sync has already
-      // been done outside the lock.
-      flushAndSyncDB();
-      updateContainerData(containerData::closeContainer);
-      clearPendingPutBlockCache();
-    } finally {
-      writeUnlock();
-    }
+    closeAndFlushIfNeeded(containerData::closeContainer);
     LOG.info("Container {} is closed with bcsId {}.",
         containerData.getContainerID(),
         containerData.getBlockCommitSequenceId());
@@ -378,6 +359,28 @@
   }
 
   /**
+   * Sync RocksDB WAL on closing of a single container.
+   *
+   * @param closer
+   * @throws StorageContainerException
+   */
+  private void closeAndFlushIfNeeded(Runnable closer)
+      throws StorageContainerException {
+    flushAndSyncDB();
+
+    writeLock();
+    try {
+      // Second sync should be a very light operation as sync has already
+      // been done outside the lock.
+      flushAndSyncDB();
+      updateContainerData(closer);
+      clearPendingPutBlockCache();
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  /**
    *
    * Must be invoked with the writeLock held.
    *
@@ -408,7 +411,7 @@
 
   private void compactDB() throws StorageContainerException {
     try {
-      try (ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
+      try (DBHandle db = BlockUtils.getDB(containerData, config)) {
         db.getStore().compactDB();
       }
     } catch (StorageContainerException ex) {
@@ -422,7 +425,7 @@
 
   private void flushAndSyncDB() throws StorageContainerException {
     try {
-      try (ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
+      try (DBHandle db = BlockUtils.getDB(containerData, config)) {
         db.getStore().flushLog(true);
         LOG.info("Container {} is synced with bcsId {}.",
             containerData.getContainerID(),
@@ -533,6 +536,11 @@
       //rewriting the yaml file with new checksum calculation.
       update(originalContainerData.getMetadata(), true);
 
+      if (containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+        // load metadata from received dump files before we try to parse kv
+        BlockUtils.loadKVContainerDataFromFiles(containerData, config);
+      }
+
       //fill in memory stat counter (keycount, byte usage)
       KeyValueContainerUtil.parseKVContainerData(containerData, config);
 
@@ -544,6 +552,10 @@
       }
       //delete all the temporary data in case of any exception.
       try {
+        if (containerData.getSchemaVersion() != null &&
+            containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+          BlockUtils.removeContainerFromDB(containerData, config);
+        }
         FileUtils.deleteDirectory(new File(containerData.getMetadataPath()));
         FileUtils.deleteDirectory(new File(containerData.getChunksPath()));
         FileUtils.deleteDirectory(
@@ -578,16 +590,18 @@
       }
 
       try {
-        compactDB();
-        // Close DB (and remove from cache) to avoid concurrent modification
-        // while packing it.
-        BlockUtils.removeDB(containerData, config);
+        if (!containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+          compactDB();
+          // Close DB (and remove from cache) to avoid concurrent modification
+          // while packing it.
+          BlockUtils.removeDB(containerData, config);
+        }
       } finally {
         readLock();
         writeUnlock();
       }
 
-      packer.pack(this, destination);
+      packContainerToDestination(destination, packer);
     } finally {
       if (lock.isWriteLockedByCurrentThread()) {
         writeUnlock();
@@ -800,8 +814,7 @@
    * @return
    */
   public File getContainerDBFile() {
-    return new File(containerData.getMetadataPath(), containerData
-        .getContainerID() + OzoneConsts.DN_CONTAINER_DB);
+    return KeyValueContainerLocationUtil.getContainerDBFile(containerData);
   }
 
   @Override
@@ -809,7 +822,7 @@
     long containerId = containerData.getContainerID();
     KeyValueContainerCheck checker =
         new KeyValueContainerCheck(containerData.getMetadataPath(), config,
-            containerId);
+            containerId, containerData.getVolume());
     return checker.fastCheck();
   }
 
@@ -830,7 +843,7 @@
     long containerId = containerData.getContainerID();
     KeyValueContainerCheck checker =
         new KeyValueContainerCheck(containerData.getMetadataPath(), config,
-            containerId);
+            containerId, containerData.getVolume());
 
     return checker.fullCheck(throttler, canceler);
   }
@@ -850,4 +863,20 @@
         file.getName(), file.getParentFile());
   }
 
+  private void packContainerToDestination(OutputStream destination,
+      ContainerPacker<KeyValueContainerData> packer)
+      throws IOException {
+    if (containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+      // Synchronize the dump and pack operation,
+      // so concurrent exports don't get dump files overwritten.
+      // We seldom got concurrent exports for a container,
+      // so it should not influence performance much.
+      synchronized (dumpLock) {
+        BlockUtils.dumpKVContainerDataToFiles(containerData, config);
+        packer.pack(this, destination);
+      }
+    } else {
+      packer.pack(this, destination);
+    }
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
index 40d527d..c560aab 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
@@ -32,10 +32,11 @@
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 
 import java.io.File;
 import java.io.IOException;
@@ -64,15 +65,17 @@
   private ConfigurationSource checkConfig;
 
   private String metadataPath;
+  private HddsVolume volume;
 
   public KeyValueContainerCheck(String metadataPath, ConfigurationSource conf,
-      long containerID) {
+      long containerID, HddsVolume volume) {
     Preconditions.checkArgument(metadataPath != null);
 
     this.checkConfig = conf;
     this.containerID = containerID;
     this.onDiskContainerData = null;
     this.metadataPath = metadataPath;
+    this.volume = volume;
   }
 
   /**
@@ -215,9 +218,8 @@
     Preconditions.checkState(onDiskContainerData != null,
         "invoke loadContainerData prior to calling this function");
 
-    File metaDir = new File(metadataPath);
     File dbFile = KeyValueContainerLocationUtil
-        .getContainerDBFile(metaDir, containerID);
+        .getContainerDBFile(onDiskContainerData);
 
     if (!dbFile.exists() || !dbFile.canRead()) {
       String dbFileErrorMsg = "Unable to access DB File [" + dbFile.toString()
@@ -230,9 +232,10 @@
 
     ContainerLayoutVersion layout = onDiskContainerData.getLayoutVersion();
 
-    try (ReferenceCountedDB db =
-            BlockUtils.getDB(onDiskContainerData, checkConfig);
-        BlockIterator<BlockData> kvIter = db.getStore().getBlockIterator()) {
+    try (DBHandle db = BlockUtils.getDB(onDiskContainerData, checkConfig);
+        BlockIterator<BlockData> kvIter = db.getStore().getBlockIterator(
+            onDiskContainerData.getContainerID(),
+            onDiskContainerData.getUnprefixedKeyFilter())) {
 
       while (kvIter.hasNext()) {
         BlockData block = kvIter.nextBlock();
@@ -242,11 +245,11 @@
 
           if (!chunkFile.exists()) {
             // concurrent mutation in Block DB? lookup the block again.
-            String localBlockID =
-                    Long.toString(block.getBlockID().getLocalID());
+            String blockKey =
+                onDiskContainerData.blockKey(block.getBlockID().getLocalID());
             BlockData bdata = db.getStore()
                     .getBlockDataTable()
-                    .get(localBlockID);
+                    .get(blockKey);
             if (bdata != null) {
               throw new IOException("Missing chunk file "
                   + chunkFile.getAbsolutePath());
@@ -329,6 +332,7 @@
 
     onDiskContainerData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);
+    onDiskContainerData.setVolume(volume);
   }
 
   private void handleCorruption(IOException e) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 0bc6c3a..4401615 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -28,11 +28,13 @@
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerDataProto;
+import org.apache.hadoop.hdds.utils.MetadataKeyFilters.KeyPrefixFilter;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.yaml.snakeyaml.nodes.Tag;
 
 
@@ -42,14 +44,18 @@
 import java.util.concurrent.atomic.AtomicLong;
 
 import static java.lang.Math.max;
+import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE_ROCKSDB;
 import static org.apache.hadoop.ozone.OzoneConsts.CHUNKS_PATH;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE;
+import static org.apache.hadoop.ozone.OzoneConsts.DELETE_TRANSACTION_KEY;
+import static org.apache.hadoop.ozone.OzoneConsts.DELETING_KEY_PREFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.METADATA_PATH;
 import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_VERSION;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_BYTES_USED;
 import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_COUNT;
 import static org.apache.hadoop.ozone.OzoneConsts.PENDING_DELETE_BLOCK_COUNT;
+import static org.apache.hadoop.ozone.container.metadata.DatanodeSchemaThreeDBDefinition.getContainerKeyPrefix;
 
 /**
  * This class represents the KeyValueContainer metadata, which is the
@@ -109,12 +115,13 @@
     this.deleteTransactionId = 0;
   }
 
-  public KeyValueContainerData(ContainerData source) {
+  public KeyValueContainerData(KeyValueContainerData source) {
     super(source);
     Preconditions.checkArgument(source.getContainerType()
         == ContainerProtos.ContainerType.KeyValueContainer);
     this.numPendingDeletionBlocks = new AtomicLong(0);
     this.deleteTransactionId = 0;
+    this.schemaVersion = source.getSchemaVersion();
   }
 
   /**
@@ -288,20 +295,21 @@
    * @param db - Reference to container DB.
    * @param batchOperation - Batch Operation to batch DB operations.
    * @param deletedBlockCount - Number of blocks deleted.
+   * @param releasedBytes - Number of bytes released.
    * @throws IOException
    */
-  public void updateAndCommitDBCounters(ReferenceCountedDB db,
+  public void updateAndCommitDBCounters(DBHandle db,
       BatchOperation batchOperation, int deletedBlockCount,
       long releasedBytes) throws IOException {
     Table<String, Long> metadataTable = db.getStore().getMetadataTable();
 
     // Set Bytes used and block count key.
-    metadataTable.putWithBatch(batchOperation, CONTAINER_BYTES_USED,
+    metadataTable.putWithBatch(batchOperation, bytesUsedKey(),
             getBytesUsed() - releasedBytes);
-    metadataTable.putWithBatch(batchOperation, BLOCK_COUNT,
+    metadataTable.putWithBatch(batchOperation, blockCountKey(),
             getBlockCount() - deletedBlockCount);
-    metadataTable.putWithBatch(batchOperation, PENDING_DELETE_BLOCK_COUNT,
-            (long)(getNumPendingDeletionBlocks() - deletedBlockCount));
+    metadataTable.putWithBatch(batchOperation, pendingDeleteBlockCountKey(),
+        getNumPendingDeletionBlocks() - deletedBlockCount);
 
     db.getStore().getBatchHandler().commitBatchOperation(batchOperation);
   }
@@ -313,4 +321,91 @@
   public void setReplicaIndex(int replicaIndex) {
     this.replicaIndex = replicaIndex;
   }
+
+  // NOTE: Below are some helper functions to format keys according
+  // to container schemas, we should use them instead of using
+  // raw const variables defined.
+
+  public String blockKey(long localID) {
+    return formatKey(Long.toString(localID));
+  }
+
+  public String deletingBlockKey(long localID) {
+    return formatKey(DELETING_KEY_PREFIX + localID);
+  }
+
+  public String deleteTxnKey(long txnID) {
+    return formatKey(Long.toString(txnID));
+  }
+
+  public String latestDeleteTxnKey() {
+    return formatKey(DELETE_TRANSACTION_KEY);
+  }
+
+  public String bcsIdKey() {
+    return formatKey(BLOCK_COMMIT_SEQUENCE_ID);
+  }
+
+  public String blockCountKey() {
+    return formatKey(BLOCK_COUNT);
+  }
+
+  public String bytesUsedKey() {
+    return formatKey(CONTAINER_BYTES_USED);
+  }
+
+  public String pendingDeleteBlockCountKey() {
+    return formatKey(PENDING_DELETE_BLOCK_COUNT);
+  }
+
+  public String deletingBlockKeyPrefix() {
+    return formatKey(DELETING_KEY_PREFIX);
+  }
+
+  public KeyPrefixFilter getUnprefixedKeyFilter() {
+    String schemaPrefix = containerPrefix();
+    return new KeyPrefixFilter().addFilter(schemaPrefix + "#", true);
+  }
+
+  public KeyPrefixFilter getDeletingBlockKeyFilter() {
+    return new KeyPrefixFilter().addFilter(deletingBlockKeyPrefix());
+  }
+
+  /**
+   * Schema v3 use a prefix as startKey,
+   * for other schemas just return null.
+   * @return
+   */
+  public String startKeyEmpty() {
+    if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      return getContainerKeyPrefix(getContainerID());
+    }
+    return null;
+  }
+
+  /**
+   * Schema v3 use containerID as key prefix,
+   * for other schemas just return null.
+   * @return
+   */
+  public String containerPrefix() {
+    if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      return getContainerKeyPrefix(getContainerID());
+    }
+    return "";
+  }
+
+  /**
+   * Format the raw key to a schema specific format key.
+   * Schema v3 use container ID as key prefix,
+   * for other schemas just return the raw key.
+   * @param key raw key
+   * @return formatted key
+   */
+  private String formatKey(String key) {
+    if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      key = getContainerKeyPrefix(getContainerID()) + key;
+    }
+    return key;
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerMetadataInspector.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerMetadataInspector.java
index 614b63d..c6395de 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerMetadataInspector.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerMetadataInspector.java
@@ -25,7 +25,6 @@
 import com.google.gson.JsonPrimitive;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.hdds.utils.db.TableIterator;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -34,6 +33,7 @@
 import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerInspector;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
+import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaThreeImpl;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaTwoImpl;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
 import org.slf4j.Logger;
@@ -204,11 +204,12 @@
 
       // Build DB metadata values.
       Table<String, Long> metadataTable = store.getMetadataTable();
-      JsonObject dBMetadata = getDBMetadataJson(metadataTable);
+      JsonObject dBMetadata = getDBMetadataJson(metadataTable, containerData);
       containerJson.add("dBMetadata", dBMetadata);
 
       // Build aggregate values.
-      JsonObject aggregates = getAggregateValues(store, schemaVersion);
+      JsonObject aggregates = getAggregateValues(store,
+          containerData, schemaVersion);
       containerJson.add("aggregates", aggregates);
 
       // Build info about chunks directory.
@@ -223,26 +224,27 @@
     return containerJson;
   }
 
-  private JsonObject getDBMetadataJson(Table<String, Long> metadataTable)
-      throws IOException {
+  private JsonObject getDBMetadataJson(Table<String, Long> metadataTable,
+      KeyValueContainerData containerData) throws IOException {
     JsonObject dBMetadata = new JsonObject();
 
     dBMetadata.addProperty(OzoneConsts.BLOCK_COUNT,
-        metadataTable.get(OzoneConsts.BLOCK_COUNT));
+        metadataTable.get(containerData.blockCountKey()));
     dBMetadata.addProperty(OzoneConsts.CONTAINER_BYTES_USED,
-        metadataTable.get(OzoneConsts.CONTAINER_BYTES_USED));
+        metadataTable.get(containerData.bytesUsedKey()));
     dBMetadata.addProperty(OzoneConsts.PENDING_DELETE_BLOCK_COUNT,
-        metadataTable.get(OzoneConsts.PENDING_DELETE_BLOCK_COUNT));
+        metadataTable.get(containerData.pendingDeleteBlockCountKey()));
     dBMetadata.addProperty(OzoneConsts.DELETE_TRANSACTION_KEY,
-        metadataTable.get(OzoneConsts.DELETE_TRANSACTION_KEY));
+        metadataTable.get(containerData.latestDeleteTxnKey()));
     dBMetadata.addProperty(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID,
-        metadataTable.get(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID));
+        metadataTable.get(containerData.bcsIdKey()));
 
     return dBMetadata;
   }
 
   private JsonObject getAggregateValues(DatanodeStore store,
-      String schemaVersion) throws IOException {
+      KeyValueContainerData containerData, String schemaVersion)
+      throws IOException {
     JsonObject aggregates = new JsonObject();
 
     long usedBytesTotal = 0;
@@ -250,8 +252,8 @@
     long pendingDeleteBlockCountTotal = 0;
     // Count normal blocks.
     try (BlockIterator<BlockData> blockIter =
-             store.getBlockIterator(
-                 MetadataKeyFilters.getUnprefixedKeyFilter())) {
+             store.getBlockIterator(containerData.getContainerID(),
+                 containerData.getUnprefixedKeyFilter())) {
 
       while (blockIter.hasNext()) {
         blockCountTotal++;
@@ -262,8 +264,8 @@
     // Count pending delete blocks.
     if (schemaVersion.equals(OzoneConsts.SCHEMA_V1)) {
       try (BlockIterator<BlockData> blockIter =
-               store.getBlockIterator(
-                   MetadataKeyFilters.getDeletingKeyFilter())) {
+               store.getBlockIterator(containerData.getContainerID(),
+                   containerData.getDeletingBlockKeyFilter())) {
 
         while (blockIter.hasNext()) {
           blockCountTotal++;
@@ -276,6 +278,11 @@
           (DatanodeStoreSchemaTwoImpl) store;
       pendingDeleteBlockCountTotal =
           countPendingDeletesSchemaV2(schemaTwoStore);
+    } else if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      DatanodeStoreSchemaThreeImpl schemaThreeStore =
+          (DatanodeStoreSchemaThreeImpl) store;
+      pendingDeleteBlockCountTotal =
+          countPendingDeletesSchemaV3(schemaThreeStore, containerData);
     } else {
       throw new IOException("Failed to process deleted blocks for unknown " +
               "container schema " + schemaVersion);
@@ -307,8 +314,8 @@
     return chunksDirectory;
   }
 
-  private boolean checkAndRepair(JsonObject parent, ContainerData containerData,
-      DatanodeStore store) {
+  private boolean checkAndRepair(JsonObject parent,
+      KeyValueContainerData containerData, DatanodeStore store) {
     JsonArray errors = new JsonArray();
     boolean passed = true;
 
@@ -334,7 +341,7 @@
       BooleanSupplier keyRepairAction = () -> {
         boolean repaired = false;
         try {
-          metadataTable.put(OzoneConsts.BLOCK_COUNT,
+          metadataTable.put(containerData.blockCountKey(),
               blockCountAggregate.getAsLong());
           repaired = true;
         } catch (IOException ex) {
@@ -369,7 +376,7 @@
       BooleanSupplier keyRepairAction = () -> {
         boolean repaired = false;
         try {
-          metadataTable.put(OzoneConsts.CONTAINER_BYTES_USED,
+          metadataTable.put(containerData.bytesUsedKey(),
               usedBytesAggregate.getAsLong());
           repaired = true;
         } catch (IOException ex) {
@@ -450,6 +457,23 @@
     return pendingDeleteBlockCountTotal;
   }
 
+  private long countPendingDeletesSchemaV3(
+      DatanodeStoreSchemaThreeImpl schemaThreeStore,
+      KeyValueContainerData containerData) throws IOException {
+    long pendingDeleteBlockCountTotal = 0;
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String,
+            DeletedBlocksTransaction>>
+            iter = schemaThreeStore.getDeleteTransactionTable()
+            .iterator(containerData.containerPrefix())) {
+      while (iter.hasNext()) {
+        DeletedBlocksTransaction delTx = iter.next().getValue();
+        pendingDeleteBlockCountTotal += delTx.getLocalIDList().size();
+      }
+      return pendingDeleteBlockCountTotal;
+    }
+  }
+
   private static long getBlockLength(BlockData block) {
     long blockLen = 0;
     List<ContainerProtos.ChunkInfo> chunkInfoList = block.getChunks();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index d0a865f..fe5b8fc 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -179,6 +179,8 @@
 
   @Override
   public void stop() {
+    chunkManager.shutdown();
+    blockManager.shutdown();
   }
 
   @Override
@@ -317,10 +319,9 @@
       HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(
           StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()),
           container.getContainerData().getMaxSize());
-      String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
       String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(
               containerVolume, clusterId);
-      container.populatePathFields(idDir, containerVolume, hddsVolumeDir);
+      container.populatePathFields(idDir, containerVolume);
     } finally {
       volumeSet.readUnlock();
     }
@@ -941,9 +942,11 @@
       final InputStream rawContainerStream,
       final TarContainerPacker packer)
       throws IOException {
+    Preconditions.checkState(originalContainerData instanceof
+        KeyValueContainerData, "Should be KeyValueContainerData instance");
 
-    KeyValueContainerData containerData =
-        new KeyValueContainerData(originalContainerData);
+    KeyValueContainerData containerData = new KeyValueContainerData(
+        (KeyValueContainerData) originalContainerData);
 
     KeyValueContainer container = new KeyValueContainer(containerData,
         conf);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
index 5b9d2f7..e555c10 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
@@ -43,8 +43,10 @@
 import org.apache.commons.compress.compressors.CompressorException;
 import org.apache.commons.compress.compressors.CompressorStreamFactory;
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaThreeImpl;
 
 import static java.util.stream.Collectors.toList;
+import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V3;
 
 /**
  * Compress/uncompress KeyValueContainer data to a tar.gz archive.
@@ -71,7 +73,7 @@
       throws IOException {
     byte[] descriptorFileContent = null;
     KeyValueContainerData containerData = container.getContainerData();
-    Path dbRoot = containerData.getDbFile().toPath();
+    Path dbRoot = getDbPath(containerData);
     Path chunksRoot = Paths.get(containerData.getChunksPath());
 
     try (InputStream decompressed = decompress(input);
@@ -159,7 +161,7 @@
     try (OutputStream compressed = compress(output);
          ArchiveOutputStream archiveOutput = tar(compressed)) {
 
-      includePath(containerData.getDbFile().toPath(), DB_DIR_NAME,
+      includePath(getDbPath(containerData), DB_DIR_NAME,
           archiveOutput);
 
       includePath(Paths.get(containerData.getChunksPath()), CHUNKS_DIR_NAME,
@@ -198,6 +200,15 @@
         "Container descriptor is missing from the container archive.");
   }
 
+  public static Path getDbPath(KeyValueContainerData containerData) {
+    if (containerData.getSchemaVersion().equals(SCHEMA_V3)) {
+      return DatanodeStoreSchemaThreeImpl.getDumpDir(
+          new File(containerData.getMetadataPath())).toPath();
+    } else {
+      return containerData.getDbFile().toPath();
+    }
+  }
+
   private byte[] readEntry(InputStream input, final long size)
       throws IOException {
     ByteArrayOutputStream output = new ByteArrayOutputStream();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
index 6339234..73881f3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
@@ -18,15 +18,20 @@
 
 package org.apache.hadoop.ozone.container.keyvalue.helpers;
 
+import java.io.File;
 import java.io.IOException;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.apache.hadoop.ozone.container.common.utils.RawDB;
 import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
@@ -34,8 +39,11 @@
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaOneImpl;
+import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaThreeImpl;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaTwoImpl;
 
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.EXPORT_CONTAINER_METADATA_FAILED;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.IMPORT_CONTAINER_METADATA_FAILED;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNABLE_TO_READ_METADATA_DB;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNKNOWN_BCSID;
@@ -52,28 +60,29 @@
   }
 
   /**
-   * Obtain a DB handler for a given container. This handler is not cached and
-   * the caller must close it after using it.
+   * Obtain a DB handler for a given container or the underlying volume.
+   * This handler is not cached and the caller must close it after using it.
    * If another thread attempts to open the same container when it is already
    * opened by this thread, the other thread will get a RocksDB exception.
-   * @param containerID The containerID
    * @param containerDBPath The absolute path to the container database folder
    * @param schemaVersion The Container Schema version
    * @param conf Configuration
+   * @param readOnly open DB in read-only mode or not
    * @return Handler to the given container.
    * @throws IOException
    */
-  public static DatanodeStore getUncachedDatanodeStore(long containerID,
+  public static DatanodeStore getUncachedDatanodeStore(
       String containerDBPath, String schemaVersion,
       ConfigurationSource conf, boolean readOnly) throws IOException {
 
     DatanodeStore store;
     if (schemaVersion.equals(OzoneConsts.SCHEMA_V1)) {
-      store = new DatanodeStoreSchemaOneImpl(conf,
-          containerID, containerDBPath, readOnly);
+      store = new DatanodeStoreSchemaOneImpl(conf, containerDBPath, readOnly);
     } else if (schemaVersion.equals(OzoneConsts.SCHEMA_V2)) {
-      store = new DatanodeStoreSchemaTwoImpl(conf,
-          containerID, containerDBPath, readOnly);
+      store = new DatanodeStoreSchemaTwoImpl(conf, containerDBPath, readOnly);
+    } else if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      store = new DatanodeStoreSchemaThreeImpl(conf, containerDBPath,
+          readOnly);
     } else {
       throw new IllegalArgumentException(
           "Unrecognized database schema version: " + schemaVersion);
@@ -94,7 +103,7 @@
   public static DatanodeStore getUncachedDatanodeStore(
       KeyValueContainerData containerData, ConfigurationSource conf,
       boolean readOnly) throws IOException {
-    return getUncachedDatanodeStore(containerData.getContainerID(),
+    return getUncachedDatanodeStore(
         containerData.getDbFile().getAbsolutePath(),
         containerData.getSchemaVersion(), conf, readOnly);
   }
@@ -110,17 +119,24 @@
    * @return DB handle.
    * @throws StorageContainerException
    */
-  public static ReferenceCountedDB getDB(KeyValueContainerData containerData,
-                                    ConfigurationSource conf) throws
-      StorageContainerException {
+  public static DBHandle getDB(KeyValueContainerData containerData,
+      ConfigurationSource conf) throws StorageContainerException {
     Preconditions.checkNotNull(containerData);
-    ContainerCache cache = ContainerCache.getInstance(conf);
-    Preconditions.checkNotNull(cache);
     Preconditions.checkNotNull(containerData.getDbFile());
+
+    String containerDBPath = containerData.getDbFile().getAbsolutePath();
     try {
-      return cache.getDB(containerData.getContainerID(), containerData
-          .getContainerDBType(), containerData.getDbFile().getAbsolutePath(),
-              containerData.getSchemaVersion(), conf);
+      if (containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+        DatanodeStoreCache cache = DatanodeStoreCache.getInstance();
+        Preconditions.checkNotNull(cache);
+        return cache.getDB(containerDBPath);
+      } else {
+        ContainerCache cache = ContainerCache.getInstance(conf);
+        Preconditions.checkNotNull(cache);
+        return cache.getDB(containerData.getContainerID(), containerData
+                .getContainerDBType(), containerDBPath,
+            containerData.getSchemaVersion(), conf);
+      }
     } catch (IOException ex) {
       onFailure(containerData.getVolume());
       String message = String.format("Error opening DB. Container:%s " +
@@ -138,6 +154,10 @@
   public static void removeDB(KeyValueContainerData container,
       ConfigurationSource conf) {
     Preconditions.checkNotNull(container);
+    Preconditions.checkNotNull(container.getDbFile());
+    Preconditions.checkState(!container.getSchemaVersion()
+        .equals(OzoneConsts.SCHEMA_V3));
+
     ContainerCache cache = ContainerCache.getInstance(conf);
     Preconditions.checkNotNull(cache);
     cache.removeDB(container.getDbFile().getAbsolutePath());
@@ -146,24 +166,33 @@
   /**
    * Shutdown all DB Handles.
    *
-   * @param cache - Cache for DB Handles.
+   * @param config
    */
-  public static void shutdownCache(ContainerCache cache)  {
-    cache.shutdownCache();
+  public static void shutdownCache(ConfigurationSource config) {
+    ContainerCache.getInstance(config).shutdownCache();
+    DatanodeStoreCache.getInstance().shutdownCache();
   }
 
   /**
    * Add a DB handler into cache.
    *
-   * @param db - DB handler.
+   * @param store - low-level DatanodeStore for DB.
    * @param containerDBPath - DB path of the container.
    * @param conf configuration.
+   * @param schemaVersion schemaVersion.
    */
-  public static void addDB(ReferenceCountedDB db, String containerDBPath,
-      ConfigurationSource conf) {
-    ContainerCache cache = ContainerCache.getInstance(conf);
-    Preconditions.checkNotNull(cache);
-    cache.addDB(containerDBPath, db);
+  public static void addDB(DatanodeStore store, String containerDBPath,
+      ConfigurationSource conf, String schemaVersion) {
+    if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      DatanodeStoreCache cache = DatanodeStoreCache.getInstance();
+      Preconditions.checkNotNull(cache);
+      cache.addDB(containerDBPath, new RawDB(store, containerDBPath));
+    } else {
+      ContainerCache cache = ContainerCache.getInstance(conf);
+      Preconditions.checkNotNull(cache);
+      cache.addDB(containerDBPath,
+          new ReferenceCountedDB(store, containerDBPath));
+    }
   }
 
   /**
@@ -207,4 +236,102 @@
               + containerBCSId + ".", UNKNOWN_BCSID);
     }
   }
+
+  /**
+   * Remove container KV metadata from per-disk db store.
+   * @param containerData
+   * @param conf
+   * @throws IOException
+   */
+  public static void removeContainerFromDB(KeyValueContainerData containerData,
+      ConfigurationSource conf) throws IOException {
+    try (DBHandle db = getDB(containerData, conf)) {
+      Preconditions.checkState(db.getStore()
+          instanceof DatanodeStoreSchemaThreeImpl);
+
+      ((DatanodeStoreSchemaThreeImpl) db.getStore()).removeKVContainerData(
+          containerData.getContainerID());
+    }
+  }
+
+  /**
+   * Dump container KV metadata to external files.
+   * @param containerData
+   * @param conf
+   * @throws StorageContainerException
+   */
+  public static void dumpKVContainerDataToFiles(
+      KeyValueContainerData containerData,
+      ConfigurationSource conf) throws IOException {
+    try (DBHandle db = getDB(containerData, conf)) {
+      Preconditions.checkState(db.getStore()
+          instanceof DatanodeStoreSchemaThreeImpl);
+
+      DatanodeStoreSchemaThreeImpl store = (DatanodeStoreSchemaThreeImpl)
+          db.getStore();
+      long containerID = containerData.getContainerID();
+      File metaDir = new File(containerData.getMetadataPath());
+      File dumpDir = DatanodeStoreSchemaThreeImpl.getDumpDir(metaDir);
+      // cleanup old files first
+      deleteAllDumpFiles(dumpDir);
+
+      try {
+        if (!dumpDir.mkdirs() && !dumpDir.exists()) {
+          throw new IOException("Failed to create dir "
+              + dumpDir.getAbsolutePath() + " for container " + containerID +
+              " to dump metadata to files");
+        }
+        store.dumpKVContainerData(containerID, dumpDir);
+      } catch (IOException e) {
+        // cleanup partially dumped files
+        deleteAllDumpFiles(dumpDir);
+        throw new StorageContainerException("Failed to dump metadata" +
+            " for container " + containerID, e,
+            EXPORT_CONTAINER_METADATA_FAILED);
+      }
+    }
+  }
+
+  /**
+   * Load container KV metadata from external files.
+   * @param containerData
+   * @param conf
+   * @throws StorageContainerException
+   */
+  public static void loadKVContainerDataFromFiles(
+      KeyValueContainerData containerData,
+      ConfigurationSource conf) throws IOException {
+    try (DBHandle db = getDB(containerData, conf)) {
+      Preconditions.checkState(db.getStore()
+          instanceof DatanodeStoreSchemaThreeImpl);
+
+      DatanodeStoreSchemaThreeImpl store = (DatanodeStoreSchemaThreeImpl)
+          db.getStore();
+      long containerID = containerData.getContainerID();
+      File metaDir = new File(containerData.getMetadataPath());
+      File dumpDir = DatanodeStoreSchemaThreeImpl.getDumpDir(metaDir);
+      try {
+        store.loadKVContainerData(dumpDir);
+      } catch (IOException e) {
+        // Don't delete unloaded or partially loaded files on failure,
+        // but delete all partially loaded metadata.
+        store.removeKVContainerData(containerID);
+        throw new StorageContainerException("Failed to load metadata " +
+            "from files for container " + containerID, e,
+            IMPORT_CONTAINER_METADATA_FAILED);
+      } finally {
+        // cleanup already loaded files all together
+        deleteAllDumpFiles(dumpDir);
+      }
+    }
+  }
+
+  public static void deleteAllDumpFiles(File dumpDir) throws IOException {
+    try {
+      FileUtils.deleteDirectory(dumpDir);
+    } catch (IOException e) {
+      throw new IOException("Failed to delete dump files under "
+          + dumpDir.getAbsolutePath(), e);
+    }
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
index dde3e2e..13a5a69 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
@@ -20,6 +20,7 @@
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.Storage;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 
 import java.io.File;
 
@@ -105,9 +106,12 @@
   /**
    * Return containerDB File.
    */
-  public static File getContainerDBFile(File containerMetaDataPath,
-      long containerID) {
-    return new File(containerMetaDataPath, containerID + OzoneConsts
-        .DN_CONTAINER_DB);
+  public static File getContainerDBFile(KeyValueContainerData containerData) {
+    if (containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+      return new File(containerData.getVolume().getDbParentDir(),
+          OzoneConsts.CONTAINER_DB_NAME);
+    }
+    return new File(containerData.getMetadataPath(),
+        containerData.getContainerID() + OzoneConsts.DN_CONTAINER_DB);
   }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index 476eeef..b23a495 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -34,11 +34,11 @@
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
 import org.apache.hadoop.ozone.container.common.utils.ContainerInspectorUtil;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaOneImpl;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaTwoImpl;
@@ -79,7 +79,7 @@
    * @param conf The configuration to use for this container.
    * @throws IOException
    */
-  public static void createContainerMetaData(long containerID,
+  public static void createContainerMetaData(
       File containerMetaDataPath, File chunksPath, File dbFile,
       String schemaVersion, ConfigurationSource conf) throws IOException {
     Preconditions.checkNotNull(containerMetaDataPath);
@@ -104,20 +104,22 @@
 
     DatanodeStore store;
     if (schemaVersion.equals(OzoneConsts.SCHEMA_V1)) {
-      store = new DatanodeStoreSchemaOneImpl(conf,
-              containerID, dbFile.getAbsolutePath(), false);
+      store = new DatanodeStoreSchemaOneImpl(conf, dbFile.getAbsolutePath(),
+          false);
     } else if (schemaVersion.equals(OzoneConsts.SCHEMA_V2)) {
-      store = new DatanodeStoreSchemaTwoImpl(conf,
-              containerID, dbFile.getAbsolutePath(), false);
+      store = new DatanodeStoreSchemaTwoImpl(conf, dbFile.getAbsolutePath(),
+          false);
+    } else if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      // We don't create per-container store for schema v3 containers,
+      // they should use per-volume db store.
+      return;
     } else {
       throw new IllegalArgumentException(
               "Unrecognized schema version for container: " + schemaVersion);
     }
 
-    ReferenceCountedDB db =
-        new ReferenceCountedDB(store, dbFile.getAbsolutePath());
     //add db handler into cache
-    BlockUtils.addDB(db, dbFile.getAbsolutePath(), conf);
+    BlockUtils.addDB(store, dbFile.getAbsolutePath(), conf, schemaVersion);
   }
 
   /**
@@ -140,8 +142,12 @@
         .getMetadataPath());
     File chunksPath = new File(containerData.getChunksPath());
 
-    // Close the DB connection and remove the DB handler from cache
-    BlockUtils.removeDB(containerData, conf);
+    if (containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+      BlockUtils.removeContainerFromDB(containerData, conf);
+    } else {
+      // Close the DB connection and remove the DB handler from cache
+      BlockUtils.removeDB(containerData, conf);
+    }
 
     // Delete the Container MetaData path.
     FileUtils.deleteDirectory(containerMetaDataPath);
@@ -165,13 +171,18 @@
       ConfigurationSource config) throws IOException {
 
     long containerID = kvContainerData.getContainerID();
-    File metadataPath = new File(kvContainerData.getMetadataPath());
 
     // Verify Checksum
     ContainerUtils.verifyChecksum(kvContainerData, config);
 
+    if (kvContainerData.getSchemaVersion() == null) {
+      // If this container has not specified a schema version, it is in the old
+      // format with one default column family.
+      kvContainerData.setSchemaVersion(OzoneConsts.SCHEMA_V1);
+    }
+
     File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
-        metadataPath, containerID);
+        kvContainerData);
     if (!dbFile.exists()) {
       LOG.error("Container DB file is missing for ContainerID {}. " +
           "Skipping loading of this container.", containerID);
@@ -180,14 +191,14 @@
     }
     kvContainerData.setDbFile(dbFile);
 
-    if (kvContainerData.getSchemaVersion() == null) {
-      // If this container has not specified a schema version, it is in the old
-      // format with one default column family.
-      kvContainerData.setSchemaVersion(OzoneConsts.SCHEMA_V1);
+    if (kvContainerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+      try (DBHandle db = BlockUtils.getDB(kvContainerData, config)) {
+        populateContainerMetadata(kvContainerData, db.getStore());
+      }
+      return;
     }
 
-    boolean isBlockMetadataSet = false;
-    ReferenceCountedDB cachedDB = null;
+    DBHandle cachedDB = null;
     DatanodeStore store = null;
     try {
       try {
@@ -205,70 +216,7 @@
             "instance was retrieved from the cache. This should only happen " +
             "in tests");
       }
-      Table<String, Long> metadataTable = store.getMetadataTable();
-
-      // Set pending deleted block count.
-      Long pendingDeleteBlockCount =
-          metadataTable.get(OzoneConsts.PENDING_DELETE_BLOCK_COUNT);
-      if (pendingDeleteBlockCount != null) {
-        kvContainerData.incrPendingDeletionBlocks(
-                pendingDeleteBlockCount);
-      } else {
-        // Set pending deleted block count.
-        MetadataKeyFilters.KeyPrefixFilter filter =
-                MetadataKeyFilters.getDeletingKeyFilter();
-        int numPendingDeletionBlocks =
-            store.getBlockDataTable()
-            .getSequentialRangeKVs(null, Integer.MAX_VALUE, filter)
-            .size();
-        kvContainerData.incrPendingDeletionBlocks(numPendingDeletionBlocks);
-      }
-
-      // Set delete transaction id.
-      Long delTxnId =
-          metadataTable.get(OzoneConsts.DELETE_TRANSACTION_KEY);
-      if (delTxnId != null) {
-        kvContainerData
-            .updateDeleteTransactionId(delTxnId);
-      }
-
-      // Set BlockCommitSequenceId.
-      Long bcsId = metadataTable.get(
-          OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID);
-      if (bcsId != null) {
-        kvContainerData
-            .updateBlockCommitSequenceId(bcsId);
-      }
-
-      // Set bytes used.
-      // commitSpace for Open Containers relies on usedBytes
-      Long bytesUsed =
-          metadataTable.get(OzoneConsts.CONTAINER_BYTES_USED);
-      if (bytesUsed != null) {
-        isBlockMetadataSet = true;
-        kvContainerData.setBytesUsed(bytesUsed);
-      }
-
-      // Set block count.
-      Long blockCount = metadataTable.get(OzoneConsts.BLOCK_COUNT);
-      if (blockCount != null) {
-        isBlockMetadataSet = true;
-        kvContainerData.setBlockCount(blockCount);
-      }
-      if (!isBlockMetadataSet) {
-        initializeUsedBytesAndBlockCount(store, kvContainerData);
-      }
-
-      // If the container is missing a chunks directory, possibly due to the
-      // bug fixed by HDDS-6235, create it here.
-      File chunksDir = new File(kvContainerData.getChunksPath());
-      if (!chunksDir.exists()) {
-        Files.createDirectories(chunksDir.toPath());
-      }
-      // Run advanced container inspection/repair operations if specified on
-      // startup. If this method is called but not as a part of startup,
-      // The inspectors will be unloaded and this will be a no-op.
-      ContainerInspectorUtil.process(kvContainerData, store);
+      populateContainerMetadata(kvContainerData, store);
     } finally {
       if (cachedDB != null) {
         // If we get a cached instance, calling close simply decrements the
@@ -289,6 +237,78 @@
     }
   }
 
+  private static void populateContainerMetadata(
+      KeyValueContainerData kvContainerData, DatanodeStore store)
+      throws IOException {
+    boolean isBlockMetadataSet = false;
+    Table<String, Long> metadataTable = store.getMetadataTable();
+
+    // Set pending deleted block count.
+    Long pendingDeleteBlockCount =
+        metadataTable.get(kvContainerData
+            .pendingDeleteBlockCountKey());
+    if (pendingDeleteBlockCount != null) {
+      kvContainerData.incrPendingDeletionBlocks(
+          pendingDeleteBlockCount);
+    } else {
+      // Set pending deleted block count.
+      MetadataKeyFilters.KeyPrefixFilter filter =
+          kvContainerData.getDeletingBlockKeyFilter();
+      int numPendingDeletionBlocks = store.getBlockDataTable()
+              .getSequentialRangeKVs(kvContainerData.startKeyEmpty(),
+                  Integer.MAX_VALUE, kvContainerData.containerPrefix(),
+                  filter).size();
+      kvContainerData.incrPendingDeletionBlocks(numPendingDeletionBlocks);
+    }
+
+    // Set delete transaction id.
+    Long delTxnId =
+        metadataTable.get(kvContainerData.latestDeleteTxnKey());
+    if (delTxnId != null) {
+      kvContainerData
+          .updateDeleteTransactionId(delTxnId);
+    }
+
+    // Set BlockCommitSequenceId.
+    Long bcsId = metadataTable.get(
+        kvContainerData.bcsIdKey());
+    if (bcsId != null) {
+      kvContainerData
+          .updateBlockCommitSequenceId(bcsId);
+    }
+
+    // Set bytes used.
+    // commitSpace for Open Containers relies on usedBytes
+    Long bytesUsed =
+        metadataTable.get(kvContainerData.bytesUsedKey());
+    if (bytesUsed != null) {
+      isBlockMetadataSet = true;
+      kvContainerData.setBytesUsed(bytesUsed);
+    }
+
+    // Set block count.
+    Long blockCount = metadataTable.get(
+        kvContainerData.blockCountKey());
+    if (blockCount != null) {
+      isBlockMetadataSet = true;
+      kvContainerData.setBlockCount(blockCount);
+    }
+    if (!isBlockMetadataSet) {
+      initializeUsedBytesAndBlockCount(store, kvContainerData);
+    }
+
+    // If the container is missing a chunks directory, possibly due to the
+    // bug fixed by HDDS-6235, create it here.
+    File chunksDir = new File(kvContainerData.getChunksPath());
+    if (!chunksDir.exists()) {
+      Files.createDirectories(chunksDir.toPath());
+    }
+    // Run advanced container inspection/repair operations if specified on
+    // startup. If this method is called but not as a part of startup,
+    // The inspectors will be unloaded and this will be a no-op.
+    ContainerInspectorUtil.process(kvContainerData, store);
+  }
+
   /**
    * Initialize bytes used and block count.
    * @param kvData
@@ -302,8 +322,8 @@
     long usedBytes = 0;
 
     try (BlockIterator<BlockData> blockIter =
-             store.getBlockIterator(
-                 MetadataKeyFilters.getUnprefixedKeyFilter())) {
+             store.getBlockIterator(kvData.getContainerID(),
+                 kvData.getUnprefixedKeyFilter())) {
 
       while (blockIter.hasNext()) {
         blockCount++;
@@ -317,8 +337,8 @@
 
     // Count all deleting blocks.
     try (BlockIterator<BlockData> blockIter =
-             store.getBlockIterator(
-                 MetadataKeyFilters.getDeletingKeyFilter())) {
+             store.getBlockIterator(kvData.getContainerID(),
+                 kvData.getDeletingBlockKeyFilter())) {
 
       while (blockIter.hasNext()) {
         blockCount++;
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
index f0a0cb0..67b38ff 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
@@ -27,14 +27,11 @@
 import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
-import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.hdds.utils.db.Table;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
-import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
@@ -117,17 +114,18 @@
         "operation.");
     Preconditions.checkState(data.getContainerID() >= 0, "Container Id " +
         "cannot be negative");
+
+    KeyValueContainerData containerData = container.getContainerData();
+
     // We are not locking the key manager since LevelDb serializes all actions
     // against a single DB. We rely on DB level locking to avoid conflicts.
-    try (ReferenceCountedDB db = BlockUtils.
-        getDB(container.getContainerData(), config)) {
+    try (DBHandle db = BlockUtils.getDB(containerData, config)) {
       // This is a post condition that acts as a hint to the user.
       // Should never fail.
       Preconditions.checkNotNull(db, DB_NULL_ERR_MSG);
 
       long bcsId = data.getBlockCommitSequenceId();
-      long containerBCSId = container.
-          getContainerData().getBlockCommitSequenceId();
+      long containerBCSId = containerData.getBlockCommitSequenceId();
 
       // default blockCommitSequenceId for any block is 0. It the putBlock
       // request is not coming via Ratis(for test scenarios), it will be 0.
@@ -160,7 +158,7 @@
         // If block exists in cache, blockCount should not be incremented.
         if (!isBlockInCache) {
           if (db.getStore().getBlockDataTable().get(
-              Long.toString(localID)) == null) {
+              containerData.blockKey(localID)) == null) {
             // Block does not exist in DB => blockCount needs to be
             // incremented when the block is added into DB.
             incrBlockCount = true;
@@ -168,10 +166,10 @@
         }
 
         db.getStore().getBlockDataTable().putWithBatch(
-            batch, Long.toString(localID), data);
+            batch, containerData.blockKey(localID), data);
         if (bcsId != 0) {
           db.getStore().getMetadataTable().putWithBatch(
-              batch, OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID, bcsId);
+              batch, containerData.bcsIdKey(), bcsId);
         }
 
         // Set Bytes used, this bytes used will be updated for every write and
@@ -181,14 +179,14 @@
         // is only used to compute the bytes used. This is done to keep the
         // current behavior and avoid DB write during write chunk operation.
         db.getStore().getMetadataTable().putWithBatch(
-            batch, OzoneConsts.CONTAINER_BYTES_USED,
-            container.getContainerData().getBytesUsed());
+            batch, containerData.bytesUsedKey(),
+            containerData.getBytesUsed());
 
         // Set Block Count for a container.
         if (incrBlockCount) {
           db.getStore().getMetadataTable().putWithBatch(
-              batch, OzoneConsts.BLOCK_COUNT,
-              container.getContainerData().getBlockCount() + 1);
+              batch, containerData.blockCountKey(),
+              containerData.getBlockCount() + 1);
         }
 
         db.getStore().getBatchHandler().commitBatchOperation(batch);
@@ -201,7 +199,7 @@
       // Increment block count and add block to pendingPutBlockCache
       // in-memory after the DB update.
       if (incrBlockCount) {
-        container.getContainerData().incrBlockCount();
+        containerData.incrBlockCount();
       }
 
       // If the Block is not in PendingPutBlockCache (and it is not endOfBlock),
@@ -247,15 +245,15 @@
     if (containerBCSId < bcsId) {
       throw new StorageContainerException(
           "Unable to find the block with bcsID " + bcsId + " .Container "
-              + container.getContainerData().getContainerID() + " bcsId is "
+              + containerData.getContainerID() + " bcsId is "
               + containerBCSId + ".", UNKNOWN_BCSID);
     }
 
-    try (ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
+    try (DBHandle db = BlockUtils.getDB(containerData, config)) {
       // This is a post condition that acts as a hint to the user.
       // Should never fail.
       Preconditions.checkNotNull(db, DB_NULL_ERR_MSG);
-      BlockData blockData = getBlockByID(db, blockID);
+      BlockData blockData = getBlockByID(db, blockID, containerData);
       long id = blockData.getBlockID().getBlockCommitSequenceId();
       if (id < bcsId) {
         throw new StorageContainerException(
@@ -279,11 +277,11 @@
       throws IOException {
     KeyValueContainerData containerData = (KeyValueContainerData) container
         .getContainerData();
-    try (ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
+    try (DBHandle db = BlockUtils.getDB(containerData, config)) {
       // This is a post condition that acts as a hint to the user.
       // Should never fail.
       Preconditions.checkNotNull(db, DB_NULL_ERR_MSG);
-      BlockData blockData = getBlockByID(db, blockID);
+      BlockData blockData = getBlockByID(db, blockID, containerData);
       return blockData.getSize();
     }
   }
@@ -326,13 +324,14 @@
       List<BlockData> result = null;
       KeyValueContainerData cData =
           (KeyValueContainerData) container.getContainerData();
-      try (ReferenceCountedDB db = BlockUtils.getDB(cData, config)) {
+      try (DBHandle db = BlockUtils.getDB(cData, config)) {
         result = new ArrayList<>();
+        String startKey = (startLocalID == -1) ? cData.startKeyEmpty()
+            : cData.blockKey(startLocalID);
         List<? extends Table.KeyValue<String, BlockData>> range =
             db.getStore().getBlockDataTable()
-                .getSequentialRangeKVs(startLocalID == -1 ? null :
-                        Long.toString(startLocalID), count,
-                    MetadataKeyFilters.getUnprefixedKeyFilter());
+                .getSequentialRangeKVs(startKey, count,
+                    cData.containerPrefix(), cData.getUnprefixedKeyFilter());
         for (Table.KeyValue<String, BlockData> entry : range) {
           result.add(entry.getValue());
         }
@@ -348,12 +347,12 @@
    */
   @Override
   public void shutdown() {
-    BlockUtils.shutdownCache(ContainerCache.getInstance(config));
+    BlockUtils.shutdownCache(config);
   }
 
-  private BlockData getBlockByID(ReferenceCountedDB db, BlockID blockID)
-      throws IOException {
-    String blockKey = Long.toString(blockID.getLocalID());
+  private BlockData getBlockByID(DBHandle db, BlockID blockID,
+      KeyValueContainerData containerData) throws IOException {
+    String blockKey = containerData.blockKey(blockID.getLocalID());
 
     BlockData blockData = db.getStore().getBlockDataTable().get(blockKey);
     if (blockData == null) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
index 9e0a6e1..85d6591 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
@@ -36,7 +36,6 @@
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
 import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
-import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
 import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
 import org.apache.hadoop.hdds.utils.BackgroundService;
 import org.apache.hadoop.hdds.utils.BackgroundTask;
@@ -48,14 +47,13 @@
 import org.apache.hadoop.ozone.container.common.impl.TopNOrderedContainerDeletionChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDeletionChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.interfaces.Handler;
 import org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
-import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
-import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaTwoImpl;
+import org.apache.hadoop.ozone.container.metadata.DeleteTransactionStore;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdds.protocol.proto
@@ -66,6 +64,7 @@
 
 import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V1;
 import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V2;
+import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V3;
 
 import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
 import org.slf4j.Logger;
@@ -278,14 +277,16 @@
       File dataDir = new File(containerData.getChunksPath());
       long startTime = Time.monotonicNow();
       // Scan container's db and get list of under deletion blocks
-      try (ReferenceCountedDB meta = BlockUtils.getDB(containerData, conf)) {
+      try (DBHandle meta = BlockUtils.getDB(containerData, conf)) {
         if (containerData.getSchemaVersion().equals(SCHEMA_V1)) {
           crr = deleteViaSchema1(meta, container, dataDir, startTime);
         } else if (containerData.getSchemaVersion().equals(SCHEMA_V2)) {
           crr = deleteViaSchema2(meta, container, dataDir, startTime);
+        } else if (containerData.getSchemaVersion().equals(SCHEMA_V3)) {
+          crr = deleteViaSchema3(meta, container, dataDir, startTime);
         } else {
           throw new UnsupportedOperationException(
-              "Only schema version 1 and schema version 2 are supported.");
+              "Only schema version 1,2,3 are supported.");
         }
         return crr;
       } finally {
@@ -304,7 +305,7 @@
     }
 
     public ContainerBackgroundTaskResult deleteViaSchema1(
-        ReferenceCountedDB meta, Container container, File dataDir,
+        DBHandle meta, Container container, File dataDir,
         long startTime) throws IOException {
       ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult();
       if (!checkDataDir(dataDir)) {
@@ -315,10 +316,12 @@
             meta.getStore().getBlockDataTable();
 
         // # of blocks to delete is throttled
-        KeyPrefixFilter filter = MetadataKeyFilters.getDeletingKeyFilter();
+        KeyPrefixFilter filter = containerData.getDeletingBlockKeyFilter();
         List<? extends Table.KeyValue<String, BlockData>> toDeleteBlocks =
             blockDataTable
-                .getSequentialRangeKVs(null, (int) blocksToDelete, filter);
+                .getSequentialRangeKVs(containerData.startKeyEmpty(),
+                    (int) blocksToDelete, containerData.containerPrefix(),
+                    filter);
         if (toDeleteBlocks.isEmpty()) {
           LOG.debug("No under deletion block found in container : {}",
               containerData.getContainerID());
@@ -394,8 +397,41 @@
     }
 
     public ContainerBackgroundTaskResult deleteViaSchema2(
-        ReferenceCountedDB meta, Container container, File dataDir,
+        DBHandle meta, Container container, File dataDir,
         long startTime) throws IOException {
+      Deleter schema2Deleter = (table, batch, tid) -> {
+        Table<Long, DeletedBlocksTransaction> delTxTable =
+            (Table<Long, DeletedBlocksTransaction>) table;
+        delTxTable.deleteWithBatch(batch, tid);
+      };
+      Table<Long, DeletedBlocksTransaction> deleteTxns =
+          ((DeleteTransactionStore<Long>) meta.getStore())
+              .getDeleteTransactionTable();
+      return deleteViaTransactionStore(
+          deleteTxns.iterator(), meta,
+          container, dataDir, startTime, schema2Deleter);
+    }
+
+    public ContainerBackgroundTaskResult deleteViaSchema3(
+        DBHandle meta, Container container, File dataDir,
+        long startTime) throws IOException {
+      Deleter schema3Deleter = (table, batch, tid) -> {
+        Table<String, DeletedBlocksTransaction> delTxTable =
+            (Table<String, DeletedBlocksTransaction>) table;
+        delTxTable.deleteWithBatch(batch, containerData.deleteTxnKey(tid));
+      };
+      Table<String, DeletedBlocksTransaction> deleteTxns =
+          ((DeleteTransactionStore<String>) meta.getStore())
+              .getDeleteTransactionTable();
+      return deleteViaTransactionStore(
+          deleteTxns.iterator(containerData.containerPrefix()), meta,
+          container, dataDir, startTime, schema3Deleter);
+    }
+
+    public ContainerBackgroundTaskResult deleteViaTransactionStore(
+        TableIterator<?, ? extends Table.KeyValue<?, DeletedBlocksTransaction>>
+            iter, DBHandle meta, Container container, File dataDir,
+        long startTime, Deleter deleter) throws IOException {
       ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult();
       if (!checkDataDir(dataDir)) {
         return crr;
@@ -403,22 +439,18 @@
       try {
         Table<String, BlockData> blockDataTable =
             meta.getStore().getBlockDataTable();
-        DatanodeStore ds = meta.getStore();
-        DatanodeStoreSchemaTwoImpl dnStoreTwoImpl =
-            (DatanodeStoreSchemaTwoImpl) ds;
-        Table<Long, DeletedBlocksTransaction>
-            deleteTxns = dnStoreTwoImpl.getDeleteTransactionTable();
+        DeleteTransactionStore<?> txnStore =
+            (DeleteTransactionStore<?>) meta.getStore();
+        Table<?, DeletedBlocksTransaction> deleteTxns =
+            txnStore.getDeleteTransactionTable();
         List<DeletedBlocksTransaction> delBlocks = new ArrayList<>();
         int numBlocks = 0;
-        try (TableIterator<Long,
-            ? extends Table.KeyValue<Long, DeletedBlocksTransaction>> iter =
-            dnStoreTwoImpl.getDeleteTransactionTable().iterator()) {
-          while (iter.hasNext() && (numBlocks < blocksToDelete)) {
-            DeletedBlocksTransaction delTx = iter.next().getValue();
-            numBlocks += delTx.getLocalIDList().size();
-            delBlocks.add(delTx);
-          }
+        while (iter.hasNext() && (numBlocks < blocksToDelete)) {
+          DeletedBlocksTransaction delTx = iter.next().getValue();
+          numBlocks += delTx.getLocalIDList().size();
+          delBlocks.add(delTx);
         }
+        iter.close();
         if (delBlocks.isEmpty()) {
           LOG.debug("No transaction found in container : {}",
               containerData.getContainerID());
@@ -441,7 +473,7 @@
         try (BatchOperation batch = meta.getStore().getBatchHandler()
             .initBatchOperation()) {
           for (DeletedBlocksTransaction delTx : delBlocks) {
-            deleteTxns.deleteWithBatch(batch, delTx.getTxID());
+            deleter.apply(deleteTxns, batch, delTx.getTxID());
             for (Long blk : delTx.getLocalIDList()) {
               String bID = blk.toString();
               meta.getStore().getBlockDataTable().deleteWithBatch(batch, bID);
@@ -487,13 +519,13 @@
       long bytesReleased = 0;
       for (DeletedBlocksTransaction entry : delBlocks) {
         for (Long blkLong : entry.getLocalIDList()) {
-          String blk = blkLong.toString();
+          String blk = containerData.blockKey(blkLong);
           BlockData blkInfo = blockDataTable.get(blk);
-          LOG.debug("Deleting block {}", blk);
+          LOG.debug("Deleting block {}", blkLong);
           if (blkInfo == null) {
             LOG.warn("Missing delete block(Container = " +
                 container.getContainerData().getContainerID() + ", Block = " +
-                blk);
+                blkLong);
             continue;
           }
           try {
@@ -501,9 +533,9 @@
             blocksDeleted++;
             bytesReleased += KeyValueContainerUtil.getBlockLength(blkInfo);
           } catch (InvalidProtocolBufferException e) {
-            LOG.error("Failed to parse block info for block {}", blk, e);
+            LOG.error("Failed to parse block info for block {}", blkLong, e);
           } catch (IOException e) {
-            LOG.error("Failed to delete files for block {}", blk, e);
+            LOG.error("Failed to delete files for block {}", blkLong, e);
           }
         }
       }
@@ -515,4 +547,9 @@
       return priority;
     }
   }
+
+  private interface Deleter {
+    void apply(Table<?, DeletedBlocksTransaction> deleteTxnsTable,
+        BatchOperation batch, long txnID) throws IOException;
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeDBDefinition.java
index 2fb1174..49bcd8e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeDBDefinition.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeDBDefinition.java
@@ -33,12 +33,17 @@
 
   private File dbDir;
 
+  private ConfigurationSource config;
+
   /**
    * @param dbPath The absolute path to the .db file corresponding to this
+   * @param config The ozone global configuration.
    * {@link DBDefinition}.
    */
-  protected AbstractDatanodeDBDefinition(String dbPath) {
+  protected AbstractDatanodeDBDefinition(String dbPath,
+      ConfigurationSource config) {
     this.dbDir = new File(dbPath);
+    this.config = config;
   }
 
   @Override
@@ -57,6 +62,10 @@
             "No location config key available for datanode databases.");
   }
 
+  public ConfigurationSource getConfig() {
+    return config;
+  }
+
   @Override
   public DBColumnFamilyDefinition[] getColumnFamilies() {
     return new DBColumnFamilyDefinition[] {getBlockDataColumnFamily(),
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java
index f9f794d..cab08dd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java
@@ -67,7 +67,6 @@
       LoggerFactory.getLogger(AbstractDatanodeStore.class);
   private DBStore store;
   private final AbstractDatanodeDBDefinition dbDef;
-  private final long containerID;
   private final ColumnFamilyOptions cfOptions;
 
   private static DatanodeDBProfile dbProfile;
@@ -79,7 +78,7 @@
    * @param config - Ozone Configuration.
    * @throws IOException - on Failure.
    */
-  protected AbstractDatanodeStore(ConfigurationSource config, long containerID,
+  protected AbstractDatanodeStore(ConfigurationSource config,
       AbstractDatanodeDBDefinition dbDef, boolean openReadOnly)
       throws IOException {
 
@@ -92,7 +91,6 @@
     cfOptions = dbProfile.getColumnFamilyOptions(config);
 
     this.dbDef = dbDef;
-    this.containerID = containerID;
     this.openReadOnly = openReadOnly;
     start(config);
   }
@@ -178,13 +176,15 @@
   }
 
   @Override
-  public BlockIterator<BlockData> getBlockIterator() {
+  public BlockIterator<BlockData> getBlockIterator(long containerID)
+      throws IOException {
     return new KeyValueBlockIterator(containerID,
             blockDataTableWithIterator.iterator());
   }
 
   @Override
-  public BlockIterator<BlockData> getBlockIterator(KeyPrefixFilter filter) {
+  public BlockIterator<BlockData> getBlockIterator(long containerID,
+      KeyPrefixFilter filter) throws IOException {
     return new KeyValueBlockIterator(containerID,
             blockDataTableWithIterator.iterator(), filter);
   }
@@ -209,6 +209,14 @@
     return dbProfile;
   }
 
+  protected AbstractDatanodeDBDefinition getDbDef() {
+    return this.dbDef;
+  }
+
+  protected Table<String, BlockData> getBlockDataTableWithIterator() {
+    return this.blockDataTableWithIterator;
+  }
+
   private static void checkTableStatus(Table<?, ?> table, String name)
           throws IOException {
     String logMessage = "Unable to get a reference to %s table. Cannot " +
@@ -228,7 +236,7 @@
    * {@link MetadataKeyFilters#getUnprefixedKeyFilter()}
    */
   @InterfaceAudience.Public
-  private static class KeyValueBlockIterator implements
+  public static class KeyValueBlockIterator implements
           BlockIterator<BlockData>, Closeable {
 
     private static final Logger LOG = LoggerFactory.getLogger(
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java
index 1382daf..f3a9196 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.container.metadata;
 
 import org.apache.hadoop.hdds.StringUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
 import org.apache.hadoop.hdds.utils.db.LongCodec;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
@@ -68,8 +69,9 @@
             ChunkInfoList.class,
             new SchemaOneChunkInfoListCodec());
 
-  public DatanodeSchemaOneDBDefinition(String dbPath) {
-    super(dbPath);
+  public DatanodeSchemaOneDBDefinition(String dbPath,
+      ConfigurationSource config) {
+    super(dbPath, config);
   }
 
   @Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java
new file mode 100644
index 0000000..ae4de8c
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.metadata;
+
+import com.google.common.primitives.Longs;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.FixedLengthStringUtils;
+import org.apache.hadoop.hdds.utils.db.LongCodec;
+import org.apache.hadoop.hdds.utils.db.FixedLengthStringCodec;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfoList;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.db.DatanodeDBProfile;
+import org.rocksdb.ColumnFamilyOptions;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DB_PROFILE;
+import static org.apache.hadoop.hdds.utils.db.DBStoreBuilder.HDDS_DEFAULT_DB_PROFILE;
+
+/**
+ * This class defines the RocksDB structure for datanode following schema
+ * version 3, where the block data, metadata, and transactions which are to be
+ * deleted are put in their own separate column families and with containerID
+ * as key prefix.
+ *
+ * Some key format illustrations for the column families:
+ * - block_data:     containerID | blockID
+ * - metadata:       containerID | #BLOCKCOUNT
+ *                   containerID | #BYTESUSED
+ *                   ...
+ * - deleted_blocks: containerID | blockID
+ * - delete_txns:    containerID | TransactionID
+ *
+ * The keys would be encoded in a fix-length encoding style in order to
+ * utilize the "Prefix Seek" feature from Rocksdb to optimize seek.
+ */
+public class DatanodeSchemaThreeDBDefinition
+    extends AbstractDatanodeDBDefinition {
+  public static final DBColumnFamilyDefinition<String, BlockData>
+      BLOCK_DATA =
+      new DBColumnFamilyDefinition<>(
+          "block_data",
+          String.class,
+          new FixedLengthStringCodec(),
+          BlockData.class,
+          new BlockDataCodec());
+
+  public static final DBColumnFamilyDefinition<String, Long>
+      METADATA =
+      new DBColumnFamilyDefinition<>(
+          "metadata",
+          String.class,
+          new FixedLengthStringCodec(),
+          Long.class,
+          new LongCodec());
+
+  public static final DBColumnFamilyDefinition<String, ChunkInfoList>
+      DELETED_BLOCKS =
+      new DBColumnFamilyDefinition<>(
+          "deleted_blocks",
+          String.class,
+          new FixedLengthStringCodec(),
+          ChunkInfoList.class,
+          new ChunkInfoListCodec());
+
+  public static final DBColumnFamilyDefinition<String, DeletedBlocksTransaction>
+      DELETE_TRANSACTION =
+      new DBColumnFamilyDefinition<>(
+          "delete_txns",
+          String.class,
+          new FixedLengthStringCodec(),
+          DeletedBlocksTransaction.class,
+          new DeletedBlocksTransactionCodec());
+
+  private static String separator = "";
+
+  public DatanodeSchemaThreeDBDefinition(String dbPath,
+      ConfigurationSource config) {
+    super(dbPath, config);
+
+    DatanodeConfiguration dc = config.getObject(DatanodeConfiguration.class);
+    setSeparator(dc.getContainerSchemaV3KeySeparator());
+
+    // Get global ColumnFamilyOptions first.
+    DatanodeDBProfile dbProfile = DatanodeDBProfile
+        .getProfile(config.getEnum(HDDS_DB_PROFILE, HDDS_DEFAULT_DB_PROFILE));
+
+    ColumnFamilyOptions cfOptions = dbProfile.getColumnFamilyOptions(config);
+    // Use prefix seek to mitigating seek overhead.
+    // See: https://github.com/facebook/rocksdb/wiki/Prefix-Seek
+    cfOptions.useFixedLengthPrefixExtractor(getContainerKeyPrefixLength());
+
+    BLOCK_DATA.setCfOptions(cfOptions);
+    METADATA.setCfOptions(cfOptions);
+    DELETED_BLOCKS.setCfOptions(cfOptions);
+    DELETE_TRANSACTION.setCfOptions(cfOptions);
+  }
+
+  @Override
+  public DBColumnFamilyDefinition[] getColumnFamilies() {
+    return new DBColumnFamilyDefinition[] {getBlockDataColumnFamily(),
+        getMetadataColumnFamily(), getDeletedBlocksColumnFamily(),
+        getDeleteTransactionsColumnFamily()};
+  }
+
+  @Override
+  public DBColumnFamilyDefinition<String, BlockData>
+      getBlockDataColumnFamily() {
+    return BLOCK_DATA;
+  }
+
+  @Override
+  public DBColumnFamilyDefinition<String, Long> getMetadataColumnFamily() {
+    return METADATA;
+  }
+
+  @Override
+  public DBColumnFamilyDefinition<String, ChunkInfoList>
+      getDeletedBlocksColumnFamily() {
+    return DELETED_BLOCKS;
+  }
+
+  public DBColumnFamilyDefinition<String, DeletedBlocksTransaction>
+      getDeleteTransactionsColumnFamily() {
+    return DELETE_TRANSACTION;
+  }
+
+  public static String getContainerKeyPrefix(long containerID) {
+    // NOTE: Rocksdb normally needs a fixed length prefix.
+    return FixedLengthStringUtils.bytes2String(Longs.toByteArray(containerID))
+        + separator;
+  }
+
+  private static int getContainerKeyPrefixLength() {
+    return FixedLengthStringUtils.string2Bytes(
+        getContainerKeyPrefix(0L)).length;
+  }
+
+  private void setSeparator(String keySeparator) {
+    separator = keySeparator;
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java
index b72bad1..8641d78 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.container.metadata;
 
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
 import org.apache.hadoop.hdds.utils.db.LongCodec;
@@ -70,8 +71,9 @@
           StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction.class,
           new DeletedBlocksTransactionCodec());
 
-  public DatanodeSchemaTwoDBDefinition(String dbPath) {
-    super(dbPath);
+  public DatanodeSchemaTwoDBDefinition(String dbPath,
+      ConfigurationSource config) {
+    super(dbPath, config);
   }
 
   @Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java
index 5a0ce7a..d48a932 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java
@@ -19,7 +19,7 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
-import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
+import org.apache.hadoop.hdds.utils.MetadataKeyFilters.KeyPrefixFilter;
 import org.apache.hadoop.hdds.utils.db.BatchOperationHandler;
 import org.apache.hadoop.hdds.utils.db.DBStore;
 import org.apache.hadoop.hdds.utils.db.Table;
@@ -87,8 +87,9 @@
 
   void compactDB() throws IOException;
 
-  BlockIterator<BlockData> getBlockIterator();
+  BlockIterator<BlockData> getBlockIterator(long containerID)
+      throws IOException;
 
-  BlockIterator<BlockData>
-      getBlockIterator(MetadataKeyFilters.KeyPrefixFilter filter);
+  BlockIterator<BlockData> getBlockIterator(long containerID,
+      KeyPrefixFilter filter) throws IOException;
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaOneImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaOneImpl.java
index b72f19e..4b514c0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaOneImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaOneImpl.java
@@ -34,10 +34,9 @@
    * @param config - Ozone Configuration.
    * @throws IOException - on Failure.
    */
-  public DatanodeStoreSchemaOneImpl(ConfigurationSource config,
-      long containerID, String dbPath, boolean openReadOnly)
-      throws IOException {
-    super(config, containerID, new DatanodeSchemaOneDBDefinition(dbPath),
+  public DatanodeStoreSchemaOneImpl(ConfigurationSource config, String dbPath,
+      boolean openReadOnly) throws IOException {
+    super(config, new DatanodeSchemaOneDBDefinition(dbPath, config),
         openReadOnly);
   }
 
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java
new file mode 100644
index 0000000..949bcc6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.metadata;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.container.metadata.DatanodeSchemaThreeDBDefinition.getContainerKeyPrefix;
+
+/**
+ * Constructs a datanode store in accordance with schema version 3, which uses
+ * three column families/tables:
+ * 1. A block data table.
+ * 2. A metadata table.
+ * 3. A Delete Transaction Table.
+ *
+ * This is different from schema version 2 from these points:
+ * - All keys have containerID as prefix.
+ * - The table 3 has String as key instead of Long since we want to use prefix.
+ */
+public class DatanodeStoreSchemaThreeImpl extends AbstractDatanodeStore
+    implements DeleteTransactionStore<String> {
+
+  public static final String DUMP_FILE_SUFFIX = ".data";
+  public static final String DUMP_DIR = "db";
+
+  private final Table<String, DeletedBlocksTransaction> deleteTransactionTable;
+
+  public DatanodeStoreSchemaThreeImpl(ConfigurationSource config,
+      String dbPath, boolean openReadOnly) throws IOException {
+    super(config, new DatanodeSchemaThreeDBDefinition(dbPath, config),
+        openReadOnly);
+    this.deleteTransactionTable = ((DatanodeSchemaThreeDBDefinition) getDbDef())
+        .getDeleteTransactionsColumnFamily().getTable(getStore());
+  }
+
+  @Override
+  public Table<String, DeletedBlocksTransaction> getDeleteTransactionTable() {
+    return this.deleteTransactionTable;
+  }
+
+  @Override
+  public BlockIterator<BlockData> getBlockIterator(long containerID)
+      throws IOException {
+    // Here we need to filter the keys with containerID as prefix
+    // and followed by metadata prefixes such as #deleting#.
+    return new KeyValueBlockIterator(containerID,
+        getBlockDataTableWithIterator()
+            .iterator(getContainerKeyPrefix(containerID)),
+        new MetadataKeyFilters.KeyPrefixFilter().addFilter(
+            getContainerKeyPrefix(containerID) + "#", true));
+  }
+
+  @Override
+  public BlockIterator<BlockData> getBlockIterator(long containerID,
+      MetadataKeyFilters.KeyPrefixFilter filter) throws IOException {
+    return new KeyValueBlockIterator(containerID,
+        getBlockDataTableWithIterator()
+            .iterator(getContainerKeyPrefix(containerID)), filter);
+  }
+
+  public void removeKVContainerData(long containerID) throws IOException {
+    String prefix = getContainerKeyPrefix(containerID);
+    try (BatchOperation batch = getBatchHandler().initBatchOperation()) {
+      getMetadataTable().deleteBatchWithPrefix(batch, prefix);
+      getBlockDataTable().deleteBatchWithPrefix(batch, prefix);
+      getDeletedBlocksTable().deleteBatchWithPrefix(batch, prefix);
+      getDeleteTransactionTable().deleteBatchWithPrefix(batch, prefix);
+      getBatchHandler().commitBatchOperation(batch);
+    }
+  }
+
+  public void dumpKVContainerData(long containerID, File dumpDir)
+      throws IOException {
+    String prefix = getContainerKeyPrefix(containerID);
+    getMetadataTable().dumpToFileWithPrefix(
+        getTableDumpFile(getMetadataTable(), dumpDir), prefix);
+    getBlockDataTable().dumpToFileWithPrefix(
+        getTableDumpFile(getBlockDataTable(), dumpDir), prefix);
+    getDeletedBlocksTable().dumpToFileWithPrefix(
+        getTableDumpFile(getDeletedBlocksTable(), dumpDir), prefix);
+    getDeleteTransactionTable().dumpToFileWithPrefix(
+        getTableDumpFile(getDeleteTransactionTable(), dumpDir),
+        prefix);
+  }
+
+  public void loadKVContainerData(File dumpDir)
+      throws IOException {
+    getMetadataTable().loadFromFile(
+        getTableDumpFile(getMetadataTable(), dumpDir));
+    getBlockDataTable().loadFromFile(
+        getTableDumpFile(getBlockDataTable(), dumpDir));
+    getDeletedBlocksTable().loadFromFile(
+        getTableDumpFile(getDeletedBlocksTable(), dumpDir));
+    getDeleteTransactionTable().loadFromFile(
+        getTableDumpFile(getDeleteTransactionTable(), dumpDir));
+  }
+
+  public static File getTableDumpFile(Table<String, ?> table,
+      File dumpDir) throws IOException {
+    return new File(dumpDir, table.getName() + DUMP_FILE_SUFFIX);
+  }
+
+  public static File getDumpDir(File metaDir) {
+    return new File(metaDir, DUMP_DIR);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaTwoImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaTwoImpl.java
index db8fe6b..f09d30e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaTwoImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaTwoImpl.java
@@ -31,7 +31,8 @@
  * 2. A metadata table.
  * 3. A Delete Transaction Table.
  */
-public class DatanodeStoreSchemaTwoImpl extends AbstractDatanodeStore {
+public class DatanodeStoreSchemaTwoImpl extends AbstractDatanodeStore
+    implements DeleteTransactionStore<Long> {
 
   private final Table<Long, DeletedBlocksTransaction>
       deleteTransactionTable;
@@ -42,15 +43,15 @@
    * @param config - Ozone Configuration.
    * @throws IOException - on Failure.
    */
-  public DatanodeStoreSchemaTwoImpl(ConfigurationSource config,
-      long containerID, String dbPath, boolean openReadOnly)
-      throws IOException {
-    super(config, containerID, new DatanodeSchemaTwoDBDefinition(dbPath),
+  public DatanodeStoreSchemaTwoImpl(ConfigurationSource config, String dbPath,
+      boolean openReadOnly) throws IOException {
+    super(config, new DatanodeSchemaTwoDBDefinition(dbPath, config),
         openReadOnly);
-    this.deleteTransactionTable = new DatanodeSchemaTwoDBDefinition(dbPath)
+    this.deleteTransactionTable = ((DatanodeSchemaTwoDBDefinition) getDbDef())
         .getDeleteTransactionsColumnFamily().getTable(getStore());
   }
 
+  @Override
   public Table<Long, DeletedBlocksTransaction> getDeleteTransactionTable() {
     return deleteTransactionTable;
   }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeTable.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeTable.java
index 2fc5576..1d35fab 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeTable.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeTable.java
@@ -23,6 +23,7 @@
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.hdds.utils.db.TableIterator;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.List;
 
@@ -78,6 +79,14 @@
   }
 
   @Override
+  public final TableIterator<KEY, ? extends KeyValue<KEY, VALUE>> iterator(
+      KEY prefix) {
+    throw new UnsupportedOperationException("Iterating tables directly is not" +
+        " supported for datanode containers due to differing schema " +
+        "version.");
+  }
+
+  @Override
   public String getName() throws IOException {
     return table.getName();
   }
@@ -109,18 +118,35 @@
 
   @Override
   public List<? extends KeyValue<KEY, VALUE>> getRangeKVs(
-          KEY startKey, int count,
+          KEY startKey, int count, KEY prefix,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException {
-    return table.getRangeKVs(startKey, count, filters);
+    return table.getRangeKVs(startKey, count, prefix, filters);
   }
 
   @Override
   public List<? extends KeyValue<KEY, VALUE>> getSequentialRangeKVs(
-          KEY startKey, int count,
+          KEY startKey, int count, KEY prefix,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException {
-    return table.getSequentialRangeKVs(startKey, count, filters);
+    return table.getSequentialRangeKVs(startKey, count, prefix, filters);
+  }
+
+  @Override
+  public void deleteBatchWithPrefix(BatchOperation batch, KEY prefix)
+      throws IOException {
+    table.deleteBatchWithPrefix(batch, prefix);
+  }
+
+  @Override
+  public void dumpToFileWithPrefix(File externalFile, KEY prefix)
+      throws IOException {
+    table.dumpToFileWithPrefix(externalFile, prefix);
+  }
+
+  @Override
+  public void loadFromFile(File externalFile) throws IOException {
+    table.loadFromFile(externalFile);
   }
 
   @Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DeleteTransactionStore.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DeleteTransactionStore.java
new file mode 100644
index 0000000..40ae1ab
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DeleteTransactionStore.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.metadata;
+
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+import org.apache.hadoop.hdds.utils.db.Table;
+
+/**
+ * Store interface for DeleteTransaction table.
+ * @param <TXN_KEY>
+ */
+public interface DeleteTransactionStore<TXN_KEY> {
+
+  Table<TXN_KEY, DeletedBlocksTransaction> getDeleteTransactionTable();
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneDeletedBlocksTable.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneDeletedBlocksTable.java
index 3f86cd2..42858c8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneDeletedBlocksTable.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneDeletedBlocksTable.java
@@ -97,7 +97,7 @@
 
   @Override
   public List<? extends KeyValue<String, ChunkInfoList>> getRangeKVs(
-          String startKey, int count,
+          String startKey, int count, String prefix,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException {
 
@@ -105,12 +105,12 @@
     // else in this schema version. Ignore any user passed prefixes that could
     // collide with this and return results that are not deleted blocks.
     return unprefix(super.getRangeKVs(prefix(startKey), count,
-            getDeletedFilter()));
+        prefix, getDeletedFilter()));
   }
 
   @Override
   public List<? extends KeyValue<String, ChunkInfoList>> getSequentialRangeKVs(
-          String startKey, int count,
+          String startKey, int count, String prefix,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException {
 
@@ -118,7 +118,7 @@
     // else in this schema version. Ignore any user passed prefixes that could
     // collide with this and return results that are not deleted blocks.
     return unprefix(super.getSequentialRangeKVs(prefix(startKey), count,
-            getDeletedFilter()));
+        prefix, getDeletedFilter()));
   }
 
   private static String prefix(String key) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index 1af9c88..df61cac 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -40,6 +40,7 @@
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
 import org.apache.hadoop.hdds.utils.HAUtils;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
@@ -54,6 +55,7 @@
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
 import org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis;
 import org.apache.hadoop.ozone.container.common.utils.ContainerInspectorUtil;
+import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
@@ -62,6 +64,7 @@
 import org.apache.hadoop.ozone.container.keyvalue.statemachine.background.BlockDeletingService;
 import org.apache.hadoop.ozone.container.replication.ReplicationServer;
 import org.apache.hadoop.ozone.container.replication.ReplicationServer.ReplicationConfig;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures.SchemaV3;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -91,6 +94,7 @@
   private final ConfigurationSource config;
   private final MutableVolumeSet volumeSet;
   private final MutableVolumeSet metaVolumeSet;
+  private final MutableVolumeSet dbVolumeSet;
   private final StorageVolumeChecker volumeChecker;
   private final ContainerSet containerSet;
   private final XceiverServerSpi writeChannel;
@@ -119,10 +123,8 @@
    * @throws IOException
    */
   public OzoneContainer(
-      DatanodeDetails datanodeDetails, ConfigurationSource
-      conf, StateContext context, CertificateClient certClient
-  )
-      throws IOException {
+      DatanodeDetails datanodeDetails, ConfigurationSource conf,
+      StateContext context, CertificateClient certClient) throws IOException {
     config = conf;
     this.datanodeDetails = datanodeDetails;
     this.context = context;
@@ -134,6 +136,13 @@
     metaVolumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf,
         context, VolumeType.META_VOLUME, volumeChecker);
 
+    dbVolumeSet = HddsServerUtil.getDatanodeDbDirs(conf).isEmpty() ? null :
+        new MutableVolumeSet(datanodeDetails.getUuidString(), conf,
+            context, VolumeType.DB_VOLUME, volumeChecker);
+    if (SchemaV3.isFinalizedAndEnabled(config)) {
+      HddsVolumeUtil.loadAllHddsVolumeDbStore(volumeSet, dbVolumeSet, LOG);
+    }
+
     containerSet = new ContainerSet();
     metadataScanner = null;
 
@@ -363,6 +372,9 @@
     volumeChecker.shutdownAndWait(0, TimeUnit.SECONDS);
     volumeSet.shutdown();
     metaVolumeSet.shutdown();
+    if (dbVolumeSet != null) {
+      dbVolumeSet.shutdown();
+    }
     blockDeletingService.shutdown();
     ContainerMetrics.remove();
   }
@@ -421,6 +433,14 @@
       nrb.addMetadataStorageReport(
           metaReports[i].getMetadataProtoBufMessage());
     }
+
+    if (dbVolumeSet != null) {
+      StorageLocationReport[] dbReports = dbVolumeSet.getStorageReport();
+      for (int i = 0; i < dbReports.length; i++) {
+        nrb.addDbStorageReport(dbReports[i].getProtoBufMessage());
+      }
+    }
+
     return nrb.build();
   }
 
@@ -437,9 +457,12 @@
     return metaVolumeSet;
   }
 
+  public MutableVolumeSet getDbVolumeSet() {
+    return dbVolumeSet;
+  }
+
   @VisibleForTesting
   StorageVolumeChecker getVolumeChecker(ConfigurationSource conf) {
     return new StorageVolumeChecker(conf, new Timer());
   }
-
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/DatanodeSchemaV3FinalizeAction.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/DatanodeSchemaV3FinalizeAction.java
new file mode 100644
index 0000000..7436284
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/DatanodeSchemaV3FinalizeAction.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.upgrade;
+
+import org.apache.hadoop.hdds.upgrade.HDDSUpgradeAction;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
+import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
+import org.apache.hadoop.ozone.upgrade.UpgradeActionHdds;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature.DATANODE_SCHEMA_V3;
+import static org.apache.hadoop.ozone.upgrade.LayoutFeature.UpgradeActionType.ON_FINALIZE;
+import static org.apache.hadoop.ozone.upgrade.UpgradeActionHdds.Component.DATANODE;
+
+/**
+ * Upgrade Action for DataNode for SCHEMA V3.
+ */
+@UpgradeActionHdds(feature = DATANODE_SCHEMA_V3, component = DATANODE,
+    type = ON_FINALIZE)
+public class DatanodeSchemaV3FinalizeAction
+    implements HDDSUpgradeAction<DatanodeStateMachine> {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DatanodeSchemaV3FinalizeAction.class);
+
+  @Override
+  public void execute(DatanodeStateMachine dsm) throws Exception {
+    LOG.info("Upgrading Datanode volume layout for Schema V3 support.");
+
+    // Load RocksDB for each HddsVolume, build the relationship between
+    // HddsVolume and DbVolume if DbVolume is configured.
+    MutableVolumeSet dataVolumeSet = dsm.getContainer().getVolumeSet();
+    MutableVolumeSet dbVolumeSet = dsm.getContainer().getDbVolumeSet();
+    Preconditions.assertNotNull(dataVolumeSet,
+        "Data Volume should not be null");
+
+    dataVolumeSet.writeLock();
+    try {
+      for (StorageVolume hddsVolume : dataVolumeSet.getVolumesList()) {
+        HddsVolume dataVolume = (HddsVolume) hddsVolume;
+        if (dataVolume.getDbParentDir() != null) {
+          // The RocksDB for this hddsVolume is already created(newly added
+          // volume case).
+          continue;
+        }
+        dataVolume.createDbStore(dbVolumeSet);
+      }
+    } finally {
+      dataVolumeSet.writeUnlock();
+    }
+    DatanodeConfiguration dcf =
+        dsm.getConf().getObject(DatanodeConfiguration.class);
+    if (!dcf.getContainerSchemaV3Enabled()) {
+      LOG.info("Schema V3 is disabled. Won't load RocksDB in upgrade.");
+      return;
+    }
+    HddsVolumeUtil.loadAllHddsVolumeDbStore(dataVolumeSet, dbVolumeSet, LOG);
+  }
+}
+
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/ScmHAFinalizeUpgradeActionDatanode.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/ScmHAFinalizeUpgradeActionDatanode.java
index b4c130c..3a830d0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/ScmHAFinalizeUpgradeActionDatanode.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/ScmHAFinalizeUpgradeActionDatanode.java
@@ -71,7 +71,7 @@
    * Upgrade the specified volume to be compatible with SCM HA layout feature.
    * @return true if the volume upgrade succeeded, false otherwise.
    */
-  public static boolean upgradeVolume(HddsVolume volume, String clusterID) {
+  public static boolean upgradeVolume(StorageVolume volume, String clusterID) {
     Preconditions.checkNotNull(clusterID, "Cannot upgrade volume with null " +
         "cluster ID");
     File hddsVolumeDir = volume.getStorageDir();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/VersionedDatanodeFeatures.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/VersionedDatanodeFeatures.java
index 3653e6c..5f52191 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/VersionedDatanodeFeatures.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/VersionedDatanodeFeatures.java
@@ -22,7 +22,7 @@
 import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
 import org.apache.hadoop.hdds.upgrade.HDDSLayoutVersionManager;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
 
 import java.io.File;
@@ -127,7 +127,7 @@
       }
     }
 
-    public static boolean upgradeVolumeIfNeeded(HddsVolume volume,
+    public static boolean upgradeVolumeIfNeeded(StorageVolume volume,
         String clusterID) {
       File clusterIDDir = new File(volume.getStorageDir(), clusterID);
       boolean needsUpgrade = isFinalized(HDDSLayoutFeature.SCM_HA) &&
@@ -142,4 +142,28 @@
       return success;
     }
   }
+
+  /**
+   * Utilities for container Schema V3 layout feature.
+   * This schema put all container metadata info into a per-disk
+   * rocksdb instance instead of a per-container instance.
+   */
+  public static class SchemaV3 {
+    public static String chooseSchemaVersion(ConfigurationSource conf) {
+      if (isFinalizedAndEnabled(conf)) {
+        return OzoneConsts.SCHEMA_V3;
+      } else {
+        return SchemaV2.chooseSchemaVersion();
+      }
+    }
+
+    public static boolean isFinalizedAndEnabled(ConfigurationSource conf) {
+      DatanodeConfiguration dcf = conf.getObject(DatanodeConfiguration.class);
+      if (isFinalized(HDDSLayoutFeature.DATANODE_SCHEMA_V3)
+          && dcf.getContainerSchemaV3Enabled()) {
+        return true;
+      }
+      return false;
+    }
+  }
 }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java
index 8254322..a60f0d5 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java
@@ -24,6 +24,7 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -33,9 +34,13 @@
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
 import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
@@ -125,4 +130,31 @@
     kvData.setState(state);
     return new KeyValueContainer(kvData, new OzoneConfiguration());
   }
+
+  public static void enableSchemaV3(OzoneConfiguration conf) {
+    DatanodeConfiguration dc = conf.getObject(DatanodeConfiguration.class);
+    dc.setContainerSchemaV3Enabled(true);
+    conf.setFromObject(dc);
+  }
+
+  public static void disableSchemaV3(OzoneConfiguration conf) {
+    DatanodeConfiguration dc = conf.getObject(DatanodeConfiguration.class);
+    dc.setContainerSchemaV3Enabled(false);
+    conf.setFromObject(dc);
+  }
+
+  public static void createDbInstancesForTestIfNeeded(
+      MutableVolumeSet hddsVolumeSet, String scmID, String clusterID,
+      ConfigurationSource conf) {
+    DatanodeConfiguration dc = conf.getObject(DatanodeConfiguration.class);
+    if (!dc.getContainerSchemaV3Enabled()) {
+      return;
+    }
+
+    for (HddsVolume volume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      StorageVolumeUtil.checkVolume(volume, scmID, clusterID, conf,
+          null, null);
+    }
+  }
 }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
index ac9cd10..91d06f4 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
@@ -33,17 +33,14 @@
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
-import org.apache.hadoop.hdds.conf.MutableConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.utils.BackgroundService;
-import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.hdds.utils.db.TableIterator;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.Checksum;
 import org.apache.hadoop.ozone.common.ChunkBuffer;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
@@ -56,12 +53,12 @@
 import org.apache.hadoop.ozone.container.common.impl.TopNOrderedContainerDeletionChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
-import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.ContainerTestVersionInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
@@ -71,6 +68,7 @@
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
 import org.apache.hadoop.ozone.container.keyvalue.statemachine.background.BlockDeletingService;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
+import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaThreeImpl;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaTwoImpl;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.container.testutils.BlockDeletingServiceTestImpl;
@@ -78,12 +76,11 @@
 import org.apache.ozone.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
 
-import static java.util.stream.Collectors.toList;
 import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric;
 
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -91,9 +88,10 @@
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
-import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_VERSIONS;
 import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V1;
 import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V2;
+import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V3;
+import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.createDbInstancesForTestIfNeeded;
 import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_BLOCK;
 import static org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask.LOG;
 import static org.mockito.ArgumentMatchers.any;
@@ -107,53 +105,31 @@
 @RunWith(Parameterized.class)
 public class TestBlockDeletingService {
 
-  private static File testRoot;
-  private static String scmId;
-  private static String clusterID;
-  private static String datanodeUuid;
-  private static MutableConfigurationSource conf;
+  private File testRoot;
+  private String scmId;
+  private String clusterID;
+  private String datanodeUuid;
+  private OzoneConfiguration conf;
 
   private final ContainerLayoutVersion layout;
   private final String schemaVersion;
   private int blockLimitPerInterval;
-  private static VolumeSet volumeSet;
+  private MutableVolumeSet volumeSet;
 
-  public TestBlockDeletingService(LayoutInfo layoutInfo) {
-    this.layout = layoutInfo.layout;
-    this.schemaVersion = layoutInfo.schemaVersion;
+  public TestBlockDeletingService(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    conf = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, conf);
   }
 
   @Parameterized.Parameters
   public static Iterable<Object[]> parameters() {
-    return LayoutInfo.layoutList.stream().map(each -> new Object[] {each})
-        .collect(toList());
+    return ContainerTestVersionInfo.versionParameters();
   }
 
-  /**
-   * Bundles test parameters for TestBlockDeletingService.
-   */
-  public static class LayoutInfo {
-    private final String schemaVersion;
-    private final ContainerLayoutVersion layout;
-
-    public LayoutInfo(String schemaVersion, ContainerLayoutVersion layout) {
-      this.schemaVersion = schemaVersion;
-      this.layout = layout;
-    }
-
-    private static List<LayoutInfo> layoutList = new ArrayList<>();
-    static {
-      for (ContainerLayoutVersion ch :
-          ContainerLayoutVersion.getAllVersions()) {
-        for (String sch : SCHEMA_VERSIONS) {
-          layoutList.add(new LayoutInfo(sch, ch));
-        }
-      }
-    }
-  }
-
-  @BeforeClass
-  public static void init() throws IOException {
+  @Before
+  public void init() throws IOException {
     testRoot = GenericTestUtils
         .getTestDir(TestBlockDeletingService.class.getSimpleName());
     if (testRoot.exists()) {
@@ -161,16 +137,17 @@
     }
     scmId = UUID.randomUUID().toString();
     clusterID = UUID.randomUUID().toString();
-    conf = new OzoneConfiguration();
     conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, testRoot.getAbsolutePath());
     conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testRoot.getAbsolutePath());
     datanodeUuid = UUID.randomUUID().toString();
-    volumeSet = new MutableVolumeSet(datanodeUuid, conf, null,
+    volumeSet = new MutableVolumeSet(datanodeUuid, scmId, conf, null,
         StorageVolume.VolumeType.DATA_VOLUME, null);
+    createDbInstancesForTestIfNeeded(volumeSet, scmId, scmId, conf);
   }
 
-  @AfterClass
-  public static void cleanup() throws IOException {
+  @After
+  public void cleanup() throws IOException {
+    BlockUtils.shutdownCache(conf);
     FileUtils.deleteDirectory(testRoot);
   }
 
@@ -218,14 +195,14 @@
       if (schemaVersion.equals(SCHEMA_V1)) {
         createPendingDeleteBlocksSchema1(numOfBlocksPerContainer, data,
             containerID, numOfChunksPerBlock, buffer, chunkManager, container);
-      } else if (schemaVersion.equals(SCHEMA_V2)) {
-        createPendingDeleteBlocksSchema2(numOfBlocksPerContainer, txnID,
-            containerID, numOfChunksPerBlock, buffer, chunkManager, container,
-            data);
+      } else if (schemaVersion.equals(SCHEMA_V2)
+          || schemaVersion.equals(SCHEMA_V3)) {
+        createPendingDeleteBlocksViaTxn(numOfBlocksPerContainer, txnID,
+            containerID, numOfChunksPerBlock, buffer, chunkManager,
+            container, data);
       } else {
         throw new UnsupportedOperationException(
-            "Only schema version 1 and schema version 2 are "
-                + "supported.");
+            "Only schema version 1,2,3 are supported.");
       }
     }
   }
@@ -236,11 +213,10 @@
       ChunkBuffer buffer, ChunkManager chunkManager,
       KeyValueContainer container) {
     BlockID blockID = null;
-    try (ReferenceCountedDB metadata = BlockUtils.getDB(data, conf)) {
+    try (DBHandle metadata = BlockUtils.getDB(data, conf)) {
       for (int j = 0; j < numOfBlocksPerContainer; j++) {
         blockID = ContainerTestHelper.getTestBlockID(containerID);
-        String deleteStateName =
-            OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID();
+        String deleteStateName = data.deletingBlockKey(blockID.getLocalID());
         BlockData kd = new BlockData(blockID);
         List<ContainerProtos.ChunkInfo> chunks = Lists.newArrayList();
         putChunksInBlock(numOfChunksPerBlock, j, chunks, buffer, chunkManager,
@@ -258,7 +234,7 @@
   }
 
   @SuppressWarnings("checkstyle:parameternumber")
-  private void createPendingDeleteBlocksSchema2(int numOfBlocksPerContainer,
+  private void createPendingDeleteBlocksViaTxn(int numOfBlocksPerContainer,
       int txnID, long containerID, int numOfChunksPerBlock, ChunkBuffer buffer,
       ChunkManager chunkManager, KeyValueContainer container,
       KeyValueContainerData data) {
@@ -271,13 +247,13 @@
       putChunksInBlock(numOfChunksPerBlock, i, chunks, buffer, chunkManager,
           container, blockID);
       kd.setChunks(chunks);
-      String bID = null;
-      try (ReferenceCountedDB metadata = BlockUtils.getDB(data, conf)) {
-        bID = blockID.getLocalID() + "";
-        metadata.getStore().getBlockDataTable().put(bID, kd);
+      try (DBHandle metadata = BlockUtils.getDB(data, conf)) {
+        String blockKey = data.blockKey(blockID.getLocalID());
+        metadata.getStore().getBlockDataTable().put(blockKey, kd);
       } catch (IOException exception) {
         LOG.info("Exception = " + exception);
-        LOG.warn("Failed to put block: " + bID + " in BlockDataTable.");
+        LOG.warn("Failed to put block: " + blockID.getLocalID()
+            + " in BlockDataTable.");
       }
       container.getContainerData().incrPendingDeletionBlocks(1);
 
@@ -294,7 +270,7 @@
 
   private void createTxn(KeyValueContainerData data, List<Long> containerBlocks,
       int txnID, long containerID) {
-    try (ReferenceCountedDB metadata = BlockUtils.getDB(data, conf)) {
+    try (DBHandle metadata = BlockUtils.getDB(data, conf)) {
       StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction dtx =
           StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction
               .newBuilder().setTxID(txnID).setContainerID(containerID)
@@ -302,10 +278,18 @@
       try (BatchOperation batch = metadata.getStore().getBatchHandler()
           .initBatchOperation()) {
         DatanodeStore ds = metadata.getStore();
-        DatanodeStoreSchemaTwoImpl dnStoreTwoImpl =
-            (DatanodeStoreSchemaTwoImpl) ds;
-        dnStoreTwoImpl.getDeleteTransactionTable()
-            .putWithBatch(batch, (long) txnID, dtx);
+
+        if (schemaVersion.equals(SCHEMA_V3)) {
+          DatanodeStoreSchemaThreeImpl dnStoreThreeImpl =
+              (DatanodeStoreSchemaThreeImpl) ds;
+          dnStoreThreeImpl.getDeleteTransactionTable()
+              .putWithBatch(batch, data.deleteTxnKey(txnID), dtx);
+        } else {
+          DatanodeStoreSchemaTwoImpl dnStoreTwoImpl =
+              (DatanodeStoreSchemaTwoImpl) ds;
+          dnStoreTwoImpl.getDeleteTransactionTable()
+              .putWithBatch(batch, (long) txnID, dtx);
+        }
         metadata.getStore().getBatchHandler().commitBatchOperation(batch);
       }
     } catch (IOException exception) {
@@ -344,16 +328,16 @@
       KeyValueContainer container, int numOfBlocksPerContainer,
       int numOfChunksPerBlock) {
     long chunkLength = 100;
-    try (ReferenceCountedDB metadata = BlockUtils.getDB(data, conf)) {
+    try (DBHandle metadata = BlockUtils.getDB(data, conf)) {
       container.getContainerData().setBlockCount(numOfBlocksPerContainer);
       // Set block count, bytes used and pending delete block count.
       metadata.getStore().getMetadataTable()
-          .put(OzoneConsts.BLOCK_COUNT, (long) numOfBlocksPerContainer);
+          .put(data.blockCountKey(), (long) numOfBlocksPerContainer);
       metadata.getStore().getMetadataTable()
-          .put(OzoneConsts.CONTAINER_BYTES_USED,
+          .put(data.bytesUsedKey(),
               chunkLength * numOfChunksPerBlock * numOfBlocksPerContainer);
       metadata.getStore().getMetadataTable()
-          .put(OzoneConsts.PENDING_DELETE_BLOCK_COUNT,
+          .put(data.pendingDeleteBlockCountKey(),
               (long) numOfBlocksPerContainer);
     } catch (IOException exception) {
       LOG.warn("Meta Data update was not successful for container: "
@@ -375,11 +359,12 @@
    * Get under deletion blocks count from DB,
    * note this info is parsed from container.db.
    */
-  private int getUnderDeletionBlocksCount(ReferenceCountedDB meta,
+  private int getUnderDeletionBlocksCount(DBHandle meta,
       KeyValueContainerData data) throws IOException {
     if (data.getSchemaVersion().equals(SCHEMA_V1)) {
       return meta.getStore().getBlockDataTable()
-          .getRangeKVs(null, 100, MetadataKeyFilters.getDeletingKeyFilter())
+          .getRangeKVs(null, 100, data.containerPrefix(),
+              data.getDeletingBlockKeyFilter())
           .size();
     } else if (data.getSchemaVersion().equals(SCHEMA_V2)) {
       int pendingBlocks = 0;
@@ -397,9 +382,26 @@
         }
       }
       return pendingBlocks;
+    } else if (data.getSchemaVersion().equals(SCHEMA_V3)) {
+      int pendingBlocks = 0;
+      DatanodeStore ds = meta.getStore();
+      DatanodeStoreSchemaThreeImpl dnStoreThreeImpl =
+          (DatanodeStoreSchemaThreeImpl) ds;
+      try (
+          TableIterator<String, ? extends Table.KeyValue<String,
+              StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction>>
+              iter = dnStoreThreeImpl.getDeleteTransactionTable()
+              .iterator(data.containerPrefix())) {
+        while (iter.hasNext()) {
+          StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction
+              delTx = iter.next().getValue();
+          pendingBlocks += delTx.getLocalIDList().size();
+        }
+      }
+      return pendingBlocks;
     } else {
       throw new UnsupportedOperationException(
-          "Only schema version 1 and schema version 2 are supported.");
+          "Only schema version 1,2,3 are supported.");
     }
   }
 
@@ -425,11 +427,10 @@
     // Ensure 1 container was created
     List<ContainerData> containerData = Lists.newArrayList();
     containerSet.listContainer(0L, 1, containerData);
-    KeyValueContainerData data = (KeyValueContainerData) containerData.get(0);
     Assert.assertEquals(1, containerData.size());
+    KeyValueContainerData data = (KeyValueContainerData) containerData.get(0);
 
-    try (ReferenceCountedDB meta = BlockUtils.getDB(
-        (KeyValueContainerData) containerData.get(0), conf)) {
+    try (DBHandle meta = BlockUtils.getDB(data, conf)) {
       Map<Long, Container<?>> containerMap = containerSet.getContainerMapCopy();
       // NOTE: this test assumes that all the container is KetValueContainer and
       // have DeleteTransactionId in KetValueContainerData. If other
@@ -447,7 +448,7 @@
       // Ensure there are 3 blocks under deletion and 0 deleted blocks
       Assert.assertEquals(3, getUnderDeletionBlocksCount(meta, data));
       Assert.assertEquals(3, meta.getStore().getMetadataTable()
-          .get(OzoneConsts.PENDING_DELETE_BLOCK_COUNT).longValue());
+          .get(data.pendingDeleteBlockCountKey()).longValue());
 
       // Container contains 3 blocks. So, space used by the container
       // should be greater than zero.
@@ -474,9 +475,9 @@
       // Check finally DB counters.
       // Not checking bytes used, as handler is a mock call.
       Assert.assertEquals(0, meta.getStore().getMetadataTable()
-          .get(OzoneConsts.PENDING_DELETE_BLOCK_COUNT).longValue());
+          .get(data.pendingDeleteBlockCountKey()).longValue());
       Assert.assertEquals(0,
-          meta.getStore().getMetadataTable().get(OzoneConsts.BLOCK_COUNT)
+          meta.getStore().getMetadataTable().get(data.blockCountKey())
               .longValue());
     }
 
@@ -560,7 +561,7 @@
     KeyValueContainer container =
         (KeyValueContainer) containerSet.getContainerIterator().next();
     KeyValueContainerData data = container.getContainerData();
-    try (ReferenceCountedDB meta = BlockUtils.getDB(data, conf)) {
+    try (DBHandle meta = BlockUtils.getDB(data, conf)) {
       LogCapturer newLog = LogCapturer.captureLogs(BackgroundService.LOG);
       GenericTestUtils.waitFor(() -> {
         try {
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestContainerCache.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestContainerCache.java
index e55d68c..a6ef461 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestContainerCache.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestContainerCache.java
@@ -54,7 +54,7 @@
   private void createContainerDB(OzoneConfiguration conf, File dbFile)
       throws Exception {
     DatanodeStore store = new DatanodeStoreSchemaTwoImpl(
-            conf, 1, dbFile.getAbsolutePath(), false);
+            conf, dbFile.getAbsolutePath(), false);
 
     // we close since the SCM pre-creates containers.
     // we will open and put Db handle into a cache when keys are being created
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStoreCache.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStoreCache.java
new file mode 100644
index 0000000..b26ed68
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStoreCache.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.apache.hadoop.ozone.container.common.utils.RawDB;
+import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
+import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaThreeImpl;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+
+/**
+ * Test DatanodeStoreCache.
+ */
+public class TestDatanodeStoreCache {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  private OzoneConfiguration conf = new OzoneConfiguration();
+
+  @Test
+  public void testBasicOperations() throws IOException {
+    DatanodeStoreCache cache = DatanodeStoreCache.getInstance();
+    String dbPath1 = folder.newFolder("basic1").getAbsolutePath();
+    String dbPath2 = folder.newFolder("basic2").getAbsolutePath();
+    DatanodeStore store1 = new DatanodeStoreSchemaThreeImpl(conf, dbPath1,
+        false);
+    DatanodeStore store2 = new DatanodeStoreSchemaThreeImpl(conf, dbPath2,
+        false);
+
+    // test normal add
+    cache.addDB(dbPath1, new RawDB(store1, dbPath1));
+    cache.addDB(dbPath2, new RawDB(store2, dbPath2));
+    Assert.assertEquals(2, cache.size());
+
+    // test duplicate add
+    cache.addDB(dbPath1, new RawDB(store1, dbPath1));
+    Assert.assertEquals(2, cache.size());
+
+    // test get, test reference the same object using ==
+    Assert.assertTrue(store1 == cache.getDB(dbPath1).getStore());
+
+    // test remove
+    cache.removeDB(dbPath1);
+    Assert.assertEquals(1, cache.size());
+
+    // test remove non-exist
+    try {
+      cache.removeDB(dbPath1);
+    } catch (Exception e) {
+      Assert.fail("Should not throw " + e);
+    }
+
+    // test shutdown
+    cache.shutdownCache();
+    Assert.assertEquals(0, cache.size());
+  }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
index 41fccb8..3942312 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -19,9 +19,10 @@
 package org.apache.hadoop.ozone.container.common;
 
 import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
-import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo;
+import org.apache.hadoop.ozone.container.keyvalue.ContainerTestVersionInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
 import org.junit.Test;
@@ -42,14 +43,19 @@
   private static final long MAXSIZE = (long) StorageUnit.GB.toBytes(5);
 
   private final ContainerLayoutVersion layout;
+  private final String schemaVersion;
+  private final OzoneConfiguration conf;
 
-  public TestKeyValueContainerData(ContainerLayoutVersion layout) {
-    this.layout = layout;
+  public TestKeyValueContainerData(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    this.conf = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, conf);
   }
 
   @Parameterized.Parameters
   public static Iterable<Object[]> parameters() {
-    return ContainerLayoutTestInfo.containerLayoutParameters();
+    return ContainerTestVersionInfo.versionParameters();
   }
 
   @Test
@@ -94,7 +100,7 @@
     kvData.incrBlockCount();
     kvData.incrPendingDeletionBlocks(1);
     kvData.setSchemaVersion(
-        VersionedDatanodeFeatures.SchemaV2.chooseSchemaVersion());
+        VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion(conf));
 
     assertEquals(state, kvData.getState());
     assertEquals(containerDBType, kvData.getContainerDBType());
@@ -109,7 +115,7 @@
     assertEquals(1, kvData.getNumPendingDeletionBlocks());
     assertEquals(pipelineId.toString(), kvData.getOriginPipelineId());
     assertEquals(datanodeId.toString(), kvData.getOriginNodeId());
-    assertEquals(VersionedDatanodeFeatures.SchemaV2.chooseSchemaVersion(),
+    assertEquals(VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion(conf),
         kvData.getSchemaVersion());
   }
 
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
index 7cee63d..6e2e865 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
@@ -22,7 +22,7 @@
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
+import org.apache.hadoop.hdds.utils.MetadataKeyFilters.KeyPrefixFilter;
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -33,10 +33,11 @@
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.ContainerTestVersionInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
@@ -52,6 +53,8 @@
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.yaml.snakeyaml.Yaml;
 
 import java.io.File;
@@ -89,6 +92,7 @@
  * {@link TestDB}, which is used by these tests to load a pre created schema
  * version 1 RocksDB instance from test resources.
  */
+@RunWith(Parameterized.class)
 public class TestSchemaOneBackwardsCompatibility {
   private OzoneConfiguration conf;
 
@@ -98,9 +102,21 @@
   @Rule
   public TemporaryFolder tempFolder = new TemporaryFolder();
 
+  public TestSchemaOneBackwardsCompatibility(String schemaVersion) {
+    this.conf = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, conf);
+  }
+
+  @Parameterized.Parameters
+  public static Iterable<Object[]> parameters() {
+    return Arrays.asList(new Object[][]{
+        {OzoneConsts.SCHEMA_V2},
+        {OzoneConsts.SCHEMA_V3}
+    });
+  }
+
   @Before
   public void setup() throws Exception {
-    conf = new OzoneConfiguration();
     TestDB testDB = new TestDB();
 
     // Copy data to the temporary folder so it can be safely modified.
@@ -130,8 +146,7 @@
    */
   @Test
   public void testDirectTableIterationDisabled() throws Exception {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    try (DBHandle refCountedDB = BlockUtils.getDB(newKvData(), conf)) {
       DatanodeStore store = refCountedDB.getStore();
 
       assertTableIteratorUnsupported(store.getMetadataTable());
@@ -159,15 +174,16 @@
    */
   @Test
   public void testBlockIteration() throws IOException {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
-      assertEquals(TestDB.NUM_DELETED_BLOCKS, countDeletedBlocks(refCountedDB));
+    KeyValueContainerData cData = newKvData();
+    try (DBHandle refCountedDB = BlockUtils.getDB(cData, conf)) {
+      assertEquals(TestDB.NUM_DELETED_BLOCKS,
+          countDeletedBlocks(refCountedDB, cData));
 
       assertEquals(TestDB.NUM_PENDING_DELETION_BLOCKS,
-              countDeletingBlocks(refCountedDB));
+          countDeletingBlocks(refCountedDB, cData));
 
       assertEquals(TestDB.KEY_COUNT - TestDB.NUM_PENDING_DELETION_BLOCKS,
-              countUnprefixedBlocks(refCountedDB));
+          countUnprefixedBlocks(refCountedDB, cData));
 
       // Test that deleted block keys do not have a visible prefix when
       // iterating.
@@ -177,14 +193,16 @@
 
       // Test rangeKVs.
       List<? extends Table.KeyValue<String, ChunkInfoList>> deletedBlocks =
-              deletedBlocksTable.getRangeKVs(null, 100);
+              deletedBlocksTable.getRangeKVs(cData.startKeyEmpty(), 100,
+                  cData.containerPrefix());
 
       for (Table.KeyValue<String, ChunkInfoList> kv: deletedBlocks) {
         assertFalse(kv.getKey().contains(prefix));
       }
 
       // Test sequentialRangeKVs.
-      deletedBlocks = deletedBlocksTable.getRangeKVs(null, 100);
+      deletedBlocks = deletedBlocksTable.getRangeKVs(cData.startKeyEmpty(),
+          100, cData.containerPrefix());
 
       for (Table.KeyValue<String, ChunkInfoList> kv: deletedBlocks) {
         assertFalse(kv.getKey().contains(prefix));
@@ -217,17 +235,18 @@
   public void testReadWithoutMetadata() throws Exception {
     // Delete metadata keys from our copy of the DB.
     // This simulates them not being there to start with.
-    try (ReferenceCountedDB db = BlockUtils.getDB(newKvData(), conf)) {
+    KeyValueContainerData cData = newKvData();
+    try (DBHandle db = BlockUtils.getDB(cData, conf)) {
       Table<String, Long> metadataTable = db.getStore().getMetadataTable();
 
-      metadataTable.delete(OzoneConsts.BLOCK_COUNT);
-      assertNull(metadataTable.get(OzoneConsts.BLOCK_COUNT));
+      metadataTable.delete(cData.blockCountKey());
+      assertNull(metadataTable.get(cData.blockCountKey()));
 
-      metadataTable.delete(OzoneConsts.CONTAINER_BYTES_USED);
-      assertNull(metadataTable.get(OzoneConsts.CONTAINER_BYTES_USED));
+      metadataTable.delete(cData.bytesUsedKey());
+      assertNull(metadataTable.get(cData.bytesUsedKey()));
 
-      metadataTable.delete(OzoneConsts.PENDING_DELETE_BLOCK_COUNT);
-      assertNull(metadataTable.get(OzoneConsts.PENDING_DELETE_BLOCK_COUNT));
+      metadataTable.delete(cData.pendingDeleteBlockCountKey());
+      assertNull(metadataTable.get(cData.pendingDeleteBlockCountKey()));
     }
 
     // Create a new container data object, and fill in its metadata by
@@ -280,16 +299,16 @@
     final long expectedRegularBlocks =
             TestDB.KEY_COUNT - numBlocksToDelete;
 
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    KeyValueContainerData cData = newKvData();
+    try (DBHandle refCountedDB = BlockUtils.getDB(cData, conf)) {
       // Test results via block iteration.
 
       assertEquals(expectedDeletingBlocks,
-              countDeletingBlocks(refCountedDB));
+              countDeletingBlocks(refCountedDB, cData));
       assertEquals(expectedDeletedBlocks,
           TestDB.NUM_DELETED_BLOCKS + numberOfBlocksDeleted);
       assertEquals(expectedRegularBlocks,
-              countUnprefixedBlocks(refCountedDB));
+              countUnprefixedBlocks(refCountedDB, cData));
 
       // Test table metadata.
       // Because the KeyValueHandler used for the block deleting service is
@@ -298,7 +317,7 @@
       Table<String, Long> metadataTable =
               refCountedDB.getStore().getMetadataTable();
       assertEquals(expectedRegularBlocks + expectedDeletingBlocks,
-              (long)metadataTable.get(OzoneConsts.BLOCK_COUNT));
+              (long)metadataTable.get(cData.blockCountKey()));
     }
   }
 
@@ -323,12 +342,13 @@
         new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet,
             metrics, c -> {
         });
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    KeyValueContainerData cData = newKvData();
+    try (DBHandle refCountedDB = BlockUtils.getDB(cData, conf)) {
       // Read blocks that were already deleted before the upgrade.
       List<? extends Table.KeyValue<String, ChunkInfoList>> deletedBlocks =
-              refCountedDB.getStore()
-                      .getDeletedBlocksTable().getRangeKVs(null, 100);
+              refCountedDB.getStore().getDeletedBlocksTable()
+                  .getRangeKVs(cData.startKeyEmpty(), 100,
+                      cData.containerPrefix());
 
       Set<String> preUpgradeBlocks = new HashSet<>();
 
@@ -374,21 +394,22 @@
 
   @Test
   public void testReadBlockData() throws Exception {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    KeyValueContainerData cData = newKvData();
+    try (DBHandle refCountedDB = BlockUtils.getDB(cData, conf)) {
       Table<String, BlockData> blockDataTable =
           refCountedDB.getStore().getBlockDataTable();
 
       // Test encoding keys and decoding database values.
       for (String blockID: TestDB.BLOCK_IDS) {
-        BlockData blockData = blockDataTable.get(blockID);
+        String blockKey = cData.blockKey(Long.parseLong(blockID));
+        BlockData blockData = blockDataTable.get(blockKey);
         Assert.assertEquals(Long.toString(blockData.getLocalID()), blockID);
       }
 
       // Test decoding keys from the database.
       List<? extends Table.KeyValue<String, BlockData>> blockKeyValues =
-          blockDataTable.getRangeKVs(null, 100,
-              MetadataKeyFilters.getUnprefixedKeyFilter());
+          blockDataTable.getRangeKVs(cData.startKeyEmpty(), 100,
+              cData.containerPrefix(), cData.getUnprefixedKeyFilter());
 
       List<String> decodedKeys = new ArrayList<>();
 
@@ -401,7 +422,7 @@
 
       // Test reading blocks with block iterator.
       try (BlockIterator<BlockData> iter =
-              refCountedDB.getStore().getBlockIterator()) {
+              refCountedDB.getStore().getBlockIterator(TestDB.CONTAINER_ID)) {
 
         List<String> iteratorBlockIDs = new ArrayList<>();
 
@@ -417,21 +438,21 @@
 
   @Test
   public void testReadDeletingBlockData() throws Exception {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    KeyValueContainerData cData = newKvData();
+    try (DBHandle refCountedDB = BlockUtils.getDB(cData, conf)) {
       Table<String, BlockData> blockDataTable =
           refCountedDB.getStore().getBlockDataTable();
 
       for (String blockID: TestDB.DELETING_BLOCK_IDS) {
-        BlockData blockData =
-            blockDataTable.get(OzoneConsts.DELETING_KEY_PREFIX + blockID);
+        String blockKey = cData.deletingBlockKey(Long.parseLong(blockID));
+        BlockData blockData = blockDataTable.get(blockKey);
         Assert.assertEquals(Long.toString(blockData.getLocalID()), blockID);
       }
 
       // Test decoding keys from the database.
       List<? extends Table.KeyValue<String, BlockData>> blockKeyValues =
-          blockDataTable.getRangeKVs(null, 100,
-              MetadataKeyFilters.getDeletingKeyFilter());
+          blockDataTable.getRangeKVs(cData.startKeyEmpty(), 100,
+              cData.containerPrefix(), cData.getDeletingBlockKeyFilter());
 
       List<String> decodedKeys = new ArrayList<>();
 
@@ -443,17 +464,17 @@
       // Apply the deleting prefix to the saved block IDs so we can compare
       // them to the retrieved keys.
       List<String> expectedKeys = TestDB.DELETING_BLOCK_IDS.stream()
-          .map(key -> OzoneConsts.DELETING_KEY_PREFIX + key)
+          .map(key -> cData.deletingBlockKey(Long.parseLong(key)))
           .collect(Collectors.toList());
 
       Assert.assertEquals(expectedKeys, decodedKeys);
 
       // Test reading deleting blocks with block iterator.
-      MetadataKeyFilters.KeyPrefixFilter filter =
-          MetadataKeyFilters.getDeletingKeyFilter();
+      KeyPrefixFilter filter = cData.getDeletingBlockKeyFilter();
 
       try (BlockIterator<BlockData> iter =
-              refCountedDB.getStore().getBlockIterator(filter)) {
+              refCountedDB.getStore().getBlockIterator(TestDB.CONTAINER_ID,
+                  filter)) {
 
         List<String> iteratorBlockIDs = new ArrayList<>();
 
@@ -469,25 +490,25 @@
 
   @Test
   public void testReadMetadata() throws Exception {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    KeyValueContainerData cData = newKvData();
+    try (DBHandle refCountedDB = BlockUtils.getDB(cData, conf)) {
       Table<String, Long> metadataTable =
           refCountedDB.getStore().getMetadataTable();
 
       Assert.assertEquals(TestDB.KEY_COUNT,
-          metadataTable.get(OzoneConsts.BLOCK_COUNT).longValue());
+          metadataTable.get(cData.blockCountKey()).longValue());
       Assert.assertEquals(TestDB.BYTES_USED,
-          metadataTable.get(OzoneConsts.CONTAINER_BYTES_USED).longValue());
+          metadataTable.get(cData.bytesUsedKey()).longValue());
       Assert.assertEquals(TestDB.NUM_PENDING_DELETION_BLOCKS,
-          metadataTable.get(OzoneConsts.PENDING_DELETE_BLOCK_COUNT)
+          metadataTable.get(cData.pendingDeleteBlockCountKey())
               .longValue());
     }
   }
 
   @Test
   public void testReadDeletedBlocks() throws Exception {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    KeyValueContainerData cData = newKvData();
+    try (DBHandle refCountedDB = BlockUtils.getDB(cData, conf)) {
       Table<String, ChunkInfoList> deletedBlocksTable =
           refCountedDB.getStore().getDeletedBlocksTable();
 
@@ -499,7 +520,8 @@
 
       // Test decoding keys from the database.
       List<? extends Table.KeyValue<String, ChunkInfoList>> chunkInfoKeyValues =
-          deletedBlocksTable.getRangeKVs(null, 100);
+          deletedBlocksTable.getRangeKVs(cData.startKeyEmpty(), 100,
+              cData.containerPrefix());
 
       List<String> decodedKeys = new ArrayList<>();
 
@@ -593,25 +615,31 @@
             kvData.getNumPendingDeletionBlocks());
   }
 
-  private int countDeletedBlocks(ReferenceCountedDB refCountedDB)
+  private int countDeletedBlocks(DBHandle refCountedDB,
+      KeyValueContainerData cData)
           throws IOException {
     return refCountedDB.getStore().getDeletedBlocksTable()
-            .getRangeKVs(null, 100,
-                    MetadataKeyFilters.getUnprefixedKeyFilter()).size();
+            .getRangeKVs(cData.startKeyEmpty(), 100,
+                cData.containerPrefix(),
+                cData.getUnprefixedKeyFilter()).size();
   }
 
-  private int countDeletingBlocks(ReferenceCountedDB refCountedDB)
+  private int countDeletingBlocks(DBHandle refCountedDB,
+      KeyValueContainerData cData)
           throws IOException {
     return refCountedDB.getStore().getBlockDataTable()
-            .getRangeKVs(null, 100,
-                    MetadataKeyFilters.getDeletingKeyFilter()).size();
+            .getRangeKVs(cData.startKeyEmpty(), 100,
+                cData.containerPrefix(),
+                cData.getDeletingBlockKeyFilter()).size();
   }
 
-  private int countUnprefixedBlocks(ReferenceCountedDB refCountedDB)
+  private int countUnprefixedBlocks(DBHandle refCountedDB,
+      KeyValueContainerData cData)
           throws IOException {
     return refCountedDB.getStore().getBlockDataTable()
-            .getRangeKVs(null, 100,
-                    MetadataKeyFilters.getUnprefixedKeyFilter()).size();
+            .getRangeKVs(cData.startKeyEmpty(), 100,
+                cData.containerPrefix(),
+                cData.getUnprefixedKeyFilter()).size();
   }
 
   /**
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaTwoBackwardsCompatibility.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaTwoBackwardsCompatibility.java
new file mode 100644
index 0000000..47fd151
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaTwoBackwardsCompatibility.java
@@ -0,0 +1,367 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
+import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
+import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
+import org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl;
+import org.apache.hadoop.ozone.container.keyvalue.impl.FilePerBlockStrategy;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
+import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaTwoImpl;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.testutils.BlockDeletingServiceTestImpl;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_COUNT;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_BYTES_USED;
+import static org.apache.hadoop.ozone.OzoneConsts.PENDING_DELETE_BLOCK_COUNT;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests processing of containers written with DB schema version 2,
+ * which stores all its data in a one-db-per-container layout.
+ * Schema version 3 will use a one-db-per-disk layout, but it
+ * should still be able to read, delete data, and update metadata for schema
+ * version 2 containers.
+ * We have a switch "hdds.datanode.container.schema.v3.enabled", so we could
+ * create a test container with it off and turn it on later to
+ * test the container operations above.
+ * <p>
+ * The functionality executed by these tests assumes that all containers will
+ * have to be closed before an upgrade, meaning that containers written with
+ * schema version 2 will only ever be encountered in their closed state.
+ * <p>
+ */
+public class TestSchemaTwoBackwardsCompatibility {
+
+  private OzoneConfiguration conf;
+  private String clusterID;
+  private String datanodeUuid;
+  private File testRoot;
+  private MutableVolumeSet volumeSet;
+  private BlockManager blockManager;
+  private ChunkManager chunkManager;
+  private ContainerSet containerSet;
+  private KeyValueHandler keyValueHandler;
+  private OzoneContainer ozoneContainer;
+
+  private static final int BLOCKS_PER_CONTAINER = 6;
+  private static final int CHUNKS_PER_BLOCK = 2;
+  private static final int DELETE_TXNS_PER_CONTAINER = 2;
+  private static final int BLOCKS_PER_TXN = 2;
+  private static final int CHUNK_LENGTH = 1024;
+  private static final byte[] SAMPLE_DATA =
+      randomAlphanumeric(1024).getBytes(UTF_8);
+
+  private static final DispatcherContext WRITE_STAGE =
+      new DispatcherContext.Builder()
+          .setStage(DispatcherContext.WriteChunkStage.WRITE_DATA).build();
+
+  private static final DispatcherContext COMMIT_STAGE =
+      new DispatcherContext.Builder()
+          .setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA).build();
+
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  @Before
+  public void setup() throws Exception {
+    testRoot = tempFolder.newFolder();
+    conf = new OzoneConfiguration();
+
+    clusterID = UUID.randomUUID().toString();
+    datanodeUuid = UUID.randomUUID().toString();
+
+    // turn off schemaV3 first
+    ContainerTestUtils.disableSchemaV3(conf);
+    conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, testRoot.getAbsolutePath());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testRoot.getAbsolutePath());
+
+    volumeSet = new MutableVolumeSet(datanodeUuid, clusterID, conf, null,
+        StorageVolume.VolumeType.DATA_VOLUME, null);
+
+    blockManager = new BlockManagerImpl(conf);
+    chunkManager = new FilePerBlockStrategy(true, blockManager, volumeSet);
+
+    containerSet = new ContainerSet();
+    keyValueHandler = new KeyValueHandler(conf, datanodeUuid,
+        containerSet, volumeSet, ContainerMetrics.create(conf), c -> { });
+    ozoneContainer = mock(OzoneContainer.class);
+    when(ozoneContainer.getContainerSet()).thenReturn(containerSet);
+    when(ozoneContainer.getWriteChannel()).thenReturn(null);
+    ContainerDispatcher dispatcher = mock(ContainerDispatcher.class);
+    when(ozoneContainer.getDispatcher()).thenReturn(dispatcher);
+    when(dispatcher.getHandler(any())).thenReturn(keyValueHandler);
+  }
+
+  @After
+  public void cleanup() {
+    BlockUtils.shutdownCache(conf);
+  }
+
+  @Test
+  public void testDBFile() throws IOException {
+    // create a container of schema v2
+    KeyValueContainer container = createTestContainer();
+    assertEquals(container.getContainerData().getSchemaVersion(),
+        OzoneConsts.SCHEMA_V2);
+
+    // db file should be under the container path
+    String containerPath = container.getContainerData().getDbFile()
+        .getParentFile().getParentFile().getName();
+    assertEquals(containerPath,
+        Long.toString(container.getContainerData().getContainerID()));
+  }
+
+  @Test
+  public void testBlockIteration() throws IOException {
+    // create a container of schema v2
+    KeyValueContainer container = createTestContainer();
+    assertEquals(container.getContainerData().getSchemaVersion(),
+        OzoneConsts.SCHEMA_V2);
+
+    // turn on schema v3 first, then do operations
+    ContainerTestUtils.enableSchemaV3(conf);
+
+    try (DBHandle db = BlockUtils.getDB(container.getContainerData(), conf)) {
+      long containerID = container.getContainerData().getContainerID();
+      int blockCount = 0;
+      try (BlockIterator<BlockData> iter = db.getStore()
+          .getBlockIterator(containerID)) {
+        while (iter.hasNext()) {
+          BlockData blockData = iter.nextBlock();
+          int chunkCount = 0;
+          for (ContainerProtos.ChunkInfo chunkInfo : blockData.getChunks()) {
+            assertEquals(chunkInfo.getLen(), CHUNK_LENGTH);
+            chunkCount++;
+          }
+          assertEquals(chunkCount, CHUNKS_PER_BLOCK);
+          blockCount++;
+        }
+      }
+      assertEquals(blockCount, BLOCKS_PER_CONTAINER);
+    }
+  }
+
+  @Test
+  public void testReadMetadata() throws IOException {
+    // create a container of schema v2
+    KeyValueContainer container = createTestContainer();
+    assertEquals(container.getContainerData().getSchemaVersion(),
+        OzoneConsts.SCHEMA_V2);
+    KeyValueContainerData cData = container.getContainerData();
+    assertEquals(cData.getBlockCount(), BLOCKS_PER_CONTAINER);
+    assertEquals(cData.getNumPendingDeletionBlocks(),
+        DELETE_TXNS_PER_CONTAINER * BLOCKS_PER_TXN);
+    assertEquals(cData.getBytesUsed(),
+        CHUNK_LENGTH * CHUNKS_PER_BLOCK * BLOCKS_PER_CONTAINER);
+
+    // turn on schema v3 first, then do operations
+    ContainerTestUtils.enableSchemaV3(conf);
+
+    try (DBHandle db = BlockUtils.getDB(cData, conf)) {
+      Table<String, Long> metadatatable = db.getStore().getMetadataTable();
+      assertEquals((long)metadatatable.get(BLOCK_COUNT),
+          BLOCKS_PER_CONTAINER);
+      assertEquals((long)metadatatable.get(PENDING_DELETE_BLOCK_COUNT),
+          DELETE_TXNS_PER_CONTAINER * BLOCKS_PER_TXN);
+      assertEquals((long)metadatatable.get(CONTAINER_BYTES_USED),
+          CHUNK_LENGTH * CHUNKS_PER_BLOCK * BLOCKS_PER_CONTAINER);
+    }
+  }
+
+  @Test
+  public void testDeleteViaTransation() throws IOException, TimeoutException,
+      InterruptedException {
+    conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
+    conf.setInt(OzoneConfigKeys.OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER,
+        BLOCKS_PER_CONTAINER);
+
+    // create a container of schema v2
+    KeyValueContainer container = createTestContainer();
+    assertEquals(container.getContainerData().getSchemaVersion(),
+        OzoneConsts.SCHEMA_V2);
+    // close it
+    container.close();
+    containerSet.addContainer(container);
+    KeyValueContainerData cData = container.getContainerData();
+
+    // turn on schema v3 first, then do operations
+    ContainerTestUtils.enableSchemaV3(conf);
+
+    // start block deleting service
+    long initialTotalSpace = cData.getBytesUsed();
+    BlockDeletingServiceTestImpl service =
+        new BlockDeletingServiceTestImpl(ozoneContainer, 1000, conf);
+    service.start();
+    GenericTestUtils.waitFor(service::isStarted, 100, 3000);
+    service.runDeletingTasks();
+    GenericTestUtils.waitFor(() -> service.getTimesOfProcessed() == 1,
+        100, 3000);
+    GenericTestUtils.waitFor(() -> cData.getBytesUsed() != initialTotalSpace,
+        100, 3000);
+
+    // check in-memory metadata after deletion
+    long blockSize = CHUNK_LENGTH * CHUNKS_PER_BLOCK;
+    long expectedKeyCount = BLOCKS_PER_CONTAINER -
+        DELETE_TXNS_PER_CONTAINER * BLOCKS_PER_TXN;
+    long expectedBytesUsed = blockSize * expectedKeyCount;
+    assertEquals(cData.getBlockCount(), expectedKeyCount);
+    assertEquals(cData.getNumPendingDeletionBlocks(), 0);
+    assertEquals(cData.getBytesUsed(), expectedBytesUsed);
+
+    // check db metadata after deletion
+    try (DBHandle db = BlockUtils.getDB(cData, conf)) {
+      Table<String, Long> metadatatable = db.getStore().getMetadataTable();
+      assertEquals((long)metadatatable.get(BLOCK_COUNT), expectedKeyCount);
+      assertEquals((long)metadatatable.get(PENDING_DELETE_BLOCK_COUNT), 0);
+      assertEquals((long)metadatatable.get(CONTAINER_BYTES_USED),
+          expectedBytesUsed);
+    }
+  }
+
+  private KeyValueContainer createTestContainer() throws IOException {
+    long containerID = ContainerTestHelper.getTestContainerID();
+    KeyValueContainerData cData = new KeyValueContainerData(containerID,
+        ContainerLayoutVersion.FILE_PER_BLOCK,
+        ContainerTestHelper.CONTAINER_MAX_SIZE,
+        UUID.randomUUID().toString(), datanodeUuid);
+    cData.setSchemaVersion(OzoneConsts.SCHEMA_V2);
+    KeyValueContainer container = new KeyValueContainer(cData, conf);
+    container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
+        clusterID);
+
+    // populate with some blocks
+    // metadata will be updated here, too
+    for (long localID = 0; localID < BLOCKS_PER_CONTAINER; localID++) {
+      BlockData blockData = createTestBlockData(localID, container);
+      blockManager.putBlock(container, blockData);
+    }
+
+    // populate with some delete txns
+    for (long txnID = 0; txnID < DELETE_TXNS_PER_CONTAINER; txnID++) {
+      long startBlockID = txnID * DELETE_TXNS_PER_CONTAINER;
+      List<Long> blocks = Arrays.asList(startBlockID, startBlockID + 1);
+      DeletedBlocksTransaction txn =
+          createTestDeleteTxn(txnID, blocks, containerID);
+      try (DBHandle db = BlockUtils.getDB(cData, conf)) {
+        try (BatchOperation batch = db.getStore().getBatchHandler()
+            .initBatchOperation()) {
+          DatanodeStoreSchemaTwoImpl dnStoreTwoImpl =
+              (DatanodeStoreSchemaTwoImpl) db.getStore();
+          dnStoreTwoImpl.getDeleteTransactionTable()
+              .putWithBatch(batch, txnID, txn);
+
+          // update delete related metadata
+          db.getStore().getMetadataTable().putWithBatch(batch,
+              cData.latestDeleteTxnKey(), txn.getTxID());
+          db.getStore().getMetadataTable().putWithBatch(batch,
+              cData.pendingDeleteBlockCountKey(),
+              cData.getNumPendingDeletionBlocks() + BLOCKS_PER_TXN);
+          db.getStore().getBatchHandler().commitBatchOperation(batch);
+
+          cData.updateDeleteTransactionId(txn.getTxID());
+          cData.incrPendingDeletionBlocks(BLOCKS_PER_TXN);
+        }
+      }
+    }
+    return container;
+  }
+
+  private BlockData createTestBlockData(long localID, Container container)
+      throws StorageContainerException {
+    long containerID = container.getContainerData().getContainerID();
+    BlockID blockID = new BlockID(containerID, localID);
+    BlockData blockData = new BlockData(blockID);
+
+    // populate with some chunks
+    for (int chunkIndex = 0; chunkIndex < CHUNKS_PER_BLOCK; chunkIndex++) {
+      ChunkInfo chunk = createTestChunkData(chunkIndex, blockID, container);
+      blockData.addChunk(chunk.getProtoBufMessage());
+    }
+
+    return blockData;
+  }
+
+  private ChunkInfo createTestChunkData(long chunkIndex,
+      BlockID blockID, Container container) throws StorageContainerException {
+    String chunkName = blockID.getLocalID() + "_chunk_" + (chunkIndex + 1);
+    ChunkBuffer chunkData = ChunkBuffer.wrap(ByteBuffer.wrap(SAMPLE_DATA));
+    ChunkInfo chunkInfo = new ChunkInfo(chunkName,
+        chunkIndex * CHUNK_LENGTH, CHUNK_LENGTH);
+    chunkManager
+        .writeChunk(container, blockID, chunkInfo, chunkData, WRITE_STAGE);
+    chunkManager
+        .writeChunk(container, blockID, chunkInfo, chunkData, COMMIT_STAGE);
+    return chunkInfo;
+  }
+
+  private DeletedBlocksTransaction createTestDeleteTxn(long txnID,
+      List<Long> blocks, long containerID) {
+    return DeletedBlocksTransaction.newBuilder().setTxID(txnID)
+        .setContainerID(containerID).addAllLocalID(blocks).setCount(0).build();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
index 44f0a7f..02b673b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
@@ -19,7 +19,7 @@
 
 import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
 import org.apache.hadoop.ozone.container.common.HDDSVolumeLayoutVersion;
-import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.ozone.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.Before;
@@ -77,15 +77,15 @@
     //Check VersionFile exists
     assertTrue(versionFile.exists());
 
-    assertEquals(storageID, HddsVolumeUtil.getStorageID(
+    assertEquals(storageID, StorageVolumeUtil.getStorageID(
         properties, versionFile));
-    assertEquals(clusterID, HddsVolumeUtil.getClusterID(
+    assertEquals(clusterID, StorageVolumeUtil.getClusterID(
         properties, versionFile, clusterID));
-    assertEquals(datanodeUUID, HddsVolumeUtil.getDatanodeUUID(
+    assertEquals(datanodeUUID, StorageVolumeUtil.getDatanodeUUID(
         properties, versionFile, datanodeUUID));
-    assertEquals(cTime, HddsVolumeUtil.getCreationTime(
+    assertEquals(cTime, StorageVolumeUtil.getCreationTime(
         properties, versionFile));
-    assertEquals(lv, HddsVolumeUtil.getLayOutVersion(
+    assertEquals(lv, StorageVolumeUtil.getLayOutVersion(
         properties, versionFile));
   }
 
@@ -93,7 +93,7 @@
   public void testIncorrectClusterId() throws IOException {
     try {
       String randomClusterID = UUID.randomUUID().toString();
-      HddsVolumeUtil.getClusterID(properties, versionFile,
+      StorageVolumeUtil.getClusterID(properties, versionFile,
           randomClusterID);
       fail("Test failure in testIncorrectClusterId");
     } catch (InconsistentStorageStateException ex) {
@@ -110,7 +110,7 @@
     properties = dnVersionFile.readFrom(versionFile);
 
     try {
-      HddsVolumeUtil.getCreationTime(properties, versionFile);
+      StorageVolumeUtil.getCreationTime(properties, versionFile);
       fail("Test failure in testVerifyCTime");
     } catch (InconsistentStorageStateException ex) {
       GenericTestUtils.assertExceptionContains("Invalid Creation time in " +
@@ -127,7 +127,7 @@
     Properties props = dnVersionFile.readFrom(versionFile);
 
     try {
-      HddsVolumeUtil.getLayOutVersion(props, versionFile);
+      StorageVolumeUtil.getLayOutVersion(props, versionFile);
       fail("Test failure in testVerifyLayOut");
     } catch (InconsistentStorageStateException ex) {
       GenericTestUtils.assertExceptionContains("Invalid layOutVersion.", ex);
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
index 71f22d4..09dd32b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
@@ -48,14 +48,13 @@
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
-import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
-import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo;
+import org.apache.hadoop.ozone.container.keyvalue.ContainerTestVersionInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
@@ -75,6 +74,8 @@
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
+
+import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.createDbInstancesForTestIfNeeded;
 import static org.junit.Assert.fail;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -103,7 +104,7 @@
   private static VolumeChoosingPolicy volumeChoosingPolicy;
 
   private ContainerSet containerSet;
-  private VolumeSet volumeSet;
+  private MutableVolumeSet volumeSet;
   private BlockManager blockManager;
   private ChunkManager chunkManager;
 
@@ -116,14 +117,17 @@
   public Timeout testTimeout = Timeout.seconds(300);
 
   private final ContainerLayoutVersion layout;
+  private final String schemaVersion;
 
-  public TestContainerPersistence(ContainerLayoutVersion layout) {
-    this.layout = layout;
+  public TestContainerPersistence(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, conf);
   }
 
   @Parameterized.Parameters
   public static Iterable<Object[]> parameters() {
-    return ContainerLayoutTestInfo.containerLayoutParameters();
+    return ContainerTestVersionInfo.versionParameters();
   }
 
   @BeforeClass
@@ -146,6 +150,7 @@
     containerSet = new ContainerSet();
     volumeSet = new MutableVolumeSet(DATANODE_UUID, conf, null,
         StorageVolume.VolumeType.DATA_VOLUME, null);
+    createDbInstancesForTestIfNeeded(volumeSet, SCM_ID, SCM_ID, conf);
     blockManager = new BlockManagerImpl(conf);
     chunkManager = ChunkManagerFactory.createChunkManager(conf, blockManager,
         null);
@@ -158,6 +163,9 @@
 
   @After
   public void cleanupDir() throws IOException {
+    // Cleanup cache
+    BlockUtils.shutdownCache(conf);
+
     // Clean up SCM metadata
     log.info("Deleting {}", hddsPath);
     FileUtils.deleteDirectory(new File(hddsPath));
@@ -220,7 +228,7 @@
     Path meta = kvData.getDbFile().toPath().getParent();
     Assert.assertTrue(meta != null && Files.exists(meta));
 
-    ReferenceCountedDB store = null;
+    DBHandle store = null;
     try {
       store = BlockUtils.getDB(kvData, conf);
       Assert.assertNotNull(store);
@@ -265,6 +273,12 @@
     Assert.assertFalse(containerSet.getContainerMapCopy()
         .containsKey(testContainerID1));
 
+    // With schema v3, we don't have a container dedicated db,
+    // so skip check the behaviors related to it.
+    if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      return;
+    }
+
     // Adding block to a deleted container should fail.
     exception.expect(StorageContainerException.class);
     exception.expectMessage("Error opening DB.");
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/TestDatanodeConfiguration.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/TestDatanodeConfiguration.java
index 5f1b0a6..2bd1f0b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/TestDatanodeConfiguration.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/TestDatanodeConfiguration.java
@@ -28,6 +28,7 @@
 import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.DISK_CHECK_MIN_GAP_DEFAULT;
 import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.DISK_CHECK_TIMEOUT_DEFAULT;
 import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.DISK_CHECK_TIMEOUT_KEY;
+import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.FAILED_DB_VOLUMES_TOLERATED_KEY;
 import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.PERIODIC_DISK_CHECK_INTERVAL_MINUTES_KEY;
 import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.PERIODIC_DISK_CHECK_INTERVAL_MINUTES_DEFAULT;
 import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.FAILED_DATA_VOLUMES_TOLERATED_KEY;
@@ -57,6 +58,8 @@
         validFailedVolumesTolerated);
     conf.setInt(FAILED_METADATA_VOLUMES_TOLERATED_KEY,
         validFailedVolumesTolerated);
+    conf.setInt(FAILED_DB_VOLUMES_TOLERATED_KEY,
+        validFailedVolumesTolerated);
     conf.setTimeDuration(DISK_CHECK_MIN_GAP_KEY,
         validDiskCheckMinGap, TimeUnit.MINUTES);
     conf.setTimeDuration(DISK_CHECK_TIMEOUT_KEY,
@@ -73,6 +76,8 @@
         subject.getFailedDataVolumesTolerated());
     assertEquals(validFailedVolumesTolerated,
         subject.getFailedMetadataVolumesTolerated());
+    assertEquals(validFailedVolumesTolerated,
+        subject.getFailedDbVolumesTolerated());
     assertEquals(validDiskCheckMinGap,
         subject.getDiskCheckMinGap().toMinutes());
     assertEquals(validDiskCheckTimeout,
@@ -95,6 +100,8 @@
         invalidFailedVolumesTolerated);
     conf.setInt(FAILED_METADATA_VOLUMES_TOLERATED_KEY,
         invalidFailedVolumesTolerated);
+    conf.setInt(FAILED_DB_VOLUMES_TOLERATED_KEY,
+        invalidFailedVolumesTolerated);
     conf.setTimeDuration(DISK_CHECK_MIN_GAP_KEY,
         invalidDiskCheckMinGap, TimeUnit.MINUTES);
     conf.setTimeDuration(DISK_CHECK_TIMEOUT_KEY,
@@ -112,6 +119,8 @@
         subject.getFailedDataVolumesTolerated());
     assertEquals(FAILED_VOLUMES_TOLERATED_DEFAULT,
         subject.getFailedMetadataVolumesTolerated());
+    assertEquals(FAILED_VOLUMES_TOLERATED_DEFAULT,
+        subject.getFailedDbVolumesTolerated());
     assertEquals(DISK_CHECK_MIN_GAP_DEFAULT,
         subject.getDiskCheckMinGap().toMillis());
     assertEquals(DISK_CHECK_TIMEOUT_DEFAULT,
@@ -135,6 +144,8 @@
         subject.getFailedDataVolumesTolerated());
     assertEquals(FAILED_VOLUMES_TOLERATED_DEFAULT,
         subject.getFailedMetadataVolumesTolerated());
+    assertEquals(FAILED_VOLUMES_TOLERATED_DEFAULT,
+        subject.getFailedDbVolumesTolerated());
     assertEquals(DISK_CHECK_MIN_GAP_DEFAULT,
         subject.getDiskCheckMinGap().toMillis());
     assertEquals(DISK_CHECK_TIMEOUT_DEFAULT,
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/utils/TestHddsVolumeUtil.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/utils/TestHddsVolumeUtil.java
new file mode 100644
index 0000000..de3fc3d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/utils/TestHddsVolumeUtil.java
@@ -0,0 +1,238 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.utils;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
+import org.apache.hadoop.ozone.container.common.volume.DbVolume;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test for {@link HddsVolumeUtil}.
+ */
+public class TestHddsVolumeUtil {
+  @Rule
+  public final TemporaryFolder tempDir = new TemporaryFolder();
+
+  private final String datanodeId = UUID.randomUUID().toString();
+  private final String clusterId = UUID.randomUUID().toString();
+  private final OzoneConfiguration conf = new OzoneConfiguration();
+  private static final int VOLUMNE_NUM = 3;
+  private MutableVolumeSet hddsVolumeSet;
+  private MutableVolumeSet dbVolumeSet;
+
+  @Before
+  public void setup() throws Exception {
+    ContainerTestUtils.enableSchemaV3(conf);
+
+    // Create hdds volumes for loadAll test.
+    File[] hddsVolumeDirs = new File[VOLUMNE_NUM];
+    StringBuilder hddsDirs = new StringBuilder();
+    for (int i = 0; i < VOLUMNE_NUM; i++) {
+      hddsVolumeDirs[i] = tempDir.newFolder();
+      hddsDirs.append(hddsVolumeDirs[i]).append(",");
+    }
+    conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, hddsDirs.toString());
+    hddsVolumeSet = new MutableVolumeSet(datanodeId, clusterId, conf, null,
+        StorageVolume.VolumeType.DATA_VOLUME, null);
+
+    // Create db volumes for format and loadAll test.
+    File[] dbVolumeDirs = new File[VOLUMNE_NUM];
+    StringBuilder dbDirs = new StringBuilder();
+    for (int i = 0; i < VOLUMNE_NUM; i++) {
+      dbVolumeDirs[i] = tempDir.newFolder();
+      dbDirs.append(dbVolumeDirs[i]).append(",");
+    }
+    conf.set(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR,
+        dbDirs.toString());
+    dbVolumeSet = new MutableVolumeSet(datanodeId, clusterId, conf, null,
+        StorageVolume.VolumeType.DB_VOLUME, null);
+  }
+
+  @After
+  public void teardown() {
+    hddsVolumeSet.shutdown();
+    dbVolumeSet.shutdown();
+  }
+
+  @Test
+  public void testLoadAllHddsVolumeDbStoreWithoutDbVolumes()
+      throws IOException {
+    // Create db instances for all HddsVolumes.
+    for (HddsVolume hddsVolume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      hddsVolume.format(clusterId);
+      hddsVolume.createWorkingDir(clusterId, null);
+    }
+
+    // Reinitialize all the volumes to simulate a DN restart.
+    reinitVolumes();
+    HddsVolumeUtil.loadAllHddsVolumeDbStore(hddsVolumeSet, null, null);
+
+    for (HddsVolume hddsVolume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      File storageIdDir = new File(new File(hddsVolume.getStorageDir(),
+          clusterId), hddsVolume.getStorageID());
+
+      // No dbVolumes given, so use the hddsVolume to store db instance.
+      assertNull(hddsVolume.getDbVolume());
+      assertEquals(storageIdDir, hddsVolume.getDbParentDir());
+    }
+  }
+
+  @Test
+  public void testLoadAllHddsVolumeDbStoreWithDbVolumes()
+      throws IOException {
+    // Initialize all DbVolumes
+    for (DbVolume dbVolume : StorageVolumeUtil.getDbVolumesList(
+        dbVolumeSet.getVolumesList())) {
+      dbVolume.format(clusterId);
+      dbVolume.createWorkingDir(clusterId, null);
+    }
+
+    // Create db instances for all HddsVolumes.
+    for (HddsVolume hddsVolume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      hddsVolume.format(clusterId);
+      hddsVolume.createWorkingDir(clusterId, dbVolumeSet);
+    }
+
+    // Reinitialize all the volumes to simulate a DN restart.
+    reinitVolumes();
+    HddsVolumeUtil.loadAllHddsVolumeDbStore(hddsVolumeSet, dbVolumeSet, null);
+
+    for (HddsVolume hddsVolume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      File storageIdDir = new File(new File(hddsVolume.getStorageDir(),
+          clusterId), hddsVolume.getStorageID());
+
+      // Should not use the hddsVolume itself
+      assertNotNull(hddsVolume.getDbVolume());
+      assertNotNull(hddsVolume.getDbParentDir());
+      assertNotEquals(storageIdDir, hddsVolume.getDbParentDir());
+    }
+  }
+
+  @Test
+  public void testNoDupDbStoreCreatedWithBadDbVolumes()
+      throws IOException {
+    // Initialize all DbVolumes
+    for (DbVolume dbVolume : StorageVolumeUtil.getDbVolumesList(
+        dbVolumeSet.getVolumesList())) {
+      dbVolume.format(clusterId);
+      dbVolume.createWorkingDir(clusterId, null);
+    }
+
+    // Create db instances for all HddsVolumes.
+    for (HddsVolume hddsVolume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      hddsVolume.format(clusterId);
+      hddsVolume.createWorkingDir(clusterId, dbVolumeSet);
+    }
+
+    // Pick a dbVolume and make it fail,
+    // we should pick a dbVolume with db instances on it,
+    // and record the affected HddsVolume storageIDs.
+    int badDbVolumeCount = 0;
+    List<String> affectedHddsVolumeIDs = new ArrayList<>();
+    File badVolumeDir = null;
+    for (DbVolume dbVolume : StorageVolumeUtil.getDbVolumesList(
+        dbVolumeSet.getVolumesList())) {
+      if (!dbVolume.getHddsVolumeIDs().isEmpty()) {
+        affectedHddsVolumeIDs.addAll(dbVolume.getHddsVolumeIDs());
+        badVolumeDir = dbVolume.getStorageDir();
+        failVolume(badVolumeDir);
+        badDbVolumeCount++;
+        break;
+      }
+    }
+    assertEquals(1, badDbVolumeCount);
+    assertFalse(affectedHddsVolumeIDs.isEmpty());
+    assertNotNull(badVolumeDir);
+
+    // Reinitialize all the volumes to simulate a DN restart.
+    reinitVolumes();
+    assertEquals(1, dbVolumeSet.getFailedVolumesList().size());
+    assertEquals(VOLUMNE_NUM - 1, dbVolumeSet.getVolumesList().size());
+    HddsVolumeUtil.loadAllHddsVolumeDbStore(hddsVolumeSet, dbVolumeSet, null);
+
+    int affectedVolumeCount = 0;
+
+    for (HddsVolume hddsVolume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      File storageIdDir = new File(new File(hddsVolume.getStorageDir(),
+          clusterId), hddsVolume.getStorageID());
+
+      // This hddsVolume itself is not failed, so we could still get it here
+      if (affectedHddsVolumeIDs.contains(hddsVolume.getStorageID())) {
+        // Should not create a duplicate db instance
+        assertFalse(storageIdDir.exists());
+        assertNull(hddsVolume.getDbVolume());
+        assertNull(hddsVolume.getDbParentDir());
+        affectedVolumeCount++;
+      } else {
+        // Should not use the hddsVolume itself
+        assertNotNull(hddsVolume.getDbVolume());
+        assertNotNull(hddsVolume.getDbParentDir());
+        assertNotEquals(storageIdDir, hddsVolume.getDbParentDir());
+      }
+    }
+    assertEquals(affectedHddsVolumeIDs.size(), affectedVolumeCount);
+  }
+
+  private void reinitVolumes() throws IOException {
+    hddsVolumeSet.shutdown();
+    dbVolumeSet.shutdown();
+
+    dbVolumeSet = new MutableVolumeSet(datanodeId, conf, null,
+        StorageVolume.VolumeType.DB_VOLUME, null);
+    hddsVolumeSet = new MutableVolumeSet(datanodeId, conf, null,
+        StorageVolume.VolumeType.DATA_VOLUME, null);
+  }
+
+  /**
+   * Fail a volume by removing the VERSION file.
+   * @param volumeDir
+   */
+  private void failVolume(File volumeDir) {
+    File versionFile = new File(volumeDir, "VERSION");
+    assertTrue(versionFile.delete());
+  }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/utils/TestStorageVolumeUtil.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/utils/TestStorageVolumeUtil.java
new file mode 100644
index 0000000..b7f1397
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/utils/TestStorageVolumeUtil.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.utils;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.fs.MockSpaceUsageCheckFactory;
+import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
+import org.apache.hadoop.ozone.container.common.volume.DbVolume;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.UUID;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/**
+ * Test for {@link StorageVolumeUtil}.
+ */
+public class TestStorageVolumeUtil {
+  @Rule
+  public final TemporaryFolder folder = new TemporaryFolder();
+
+  private static final String DATANODE_UUID = UUID.randomUUID().toString();
+  private static final String CLUSTER_ID = UUID.randomUUID().toString();
+  private static final OzoneConfiguration CONF = new OzoneConfiguration();
+
+  private HddsVolume.Builder hddsVolumeBuilder;
+  private DbVolume.Builder dbVolumeBuilder;
+
+  @Before
+  public void setup() throws Exception {
+    hddsVolumeBuilder = new HddsVolume.Builder(folder.newFolder().getPath())
+        .datanodeUuid(DATANODE_UUID)
+        .conf(CONF)
+        .usageCheckFactory(MockSpaceUsageCheckFactory.NONE);
+    dbVolumeBuilder = new DbVolume.Builder(folder.newFolder().getPath())
+        .datanodeUuid(DATANODE_UUID)
+        .conf(CONF)
+        .usageCheckFactory(MockSpaceUsageCheckFactory.NONE);
+  }
+
+  @Test
+  public void testCheckVolumeNoDupDbStoreCreated() throws IOException {
+    ContainerTestUtils.enableSchemaV3(CONF);
+
+    HddsVolume hddsVolume = hddsVolumeBuilder.build();
+    HddsVolume spyHddsVolume = spy(hddsVolume);
+    DbVolume dbVolume = dbVolumeBuilder.build();
+    MutableVolumeSet dbVolumeSet = mock(MutableVolumeSet.class);
+    when(dbVolumeSet.getVolumesList())
+        .thenReturn(Collections.singletonList(dbVolume));
+
+    // check the dbVolume first for hddsVolume to use
+    boolean res = StorageVolumeUtil.checkVolume(dbVolume, CLUSTER_ID,
+        CLUSTER_ID, CONF, null, null);
+    assertTrue(res);
+
+    // checkVolume for the 1st time: rootFiles.length == 1
+    res = StorageVolumeUtil.checkVolume(spyHddsVolume, CLUSTER_ID,
+        CLUSTER_ID, CONF, null, dbVolumeSet);
+    assertTrue(res);
+    // createDbStore called as expected
+    verify(spyHddsVolume, times(1)).createDbStore(dbVolumeSet);
+
+    // checkVolume for the 2nd time: rootFiles.length == 2
+    res = StorageVolumeUtil.checkVolume(spyHddsVolume, CLUSTER_ID,
+        CLUSTER_ID, CONF, null, dbVolumeSet);
+    assertTrue(res);
+
+    // should only call createDbStore once, so no dup db instance
+    verify(spyHddsVolume, times(1)).createDbStore(dbVolumeSet);
+  }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestDbVolume.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestDbVolume.java
new file mode 100644
index 0000000..b0f0821
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestDbVolume.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.fs.MockSpaceUsageCheckFactory;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/**
+ * Unit tests for {@link DbVolume}.
+ */
+public class TestDbVolume {
+
+  private static final String DATANODE_UUID = UUID.randomUUID().toString();
+  private static final String CLUSTER_ID = UUID.randomUUID().toString();
+  private static final OzoneConfiguration CONF = new OzoneConfiguration();
+
+  private DbVolume.Builder volumeBuilder;
+  private File versionFile;
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Before
+  public void setup() throws Exception {
+    File rootDir = new File(folder.getRoot(), DbVolume.DB_VOLUME_DIR);
+    volumeBuilder = new DbVolume.Builder(folder.getRoot().getPath())
+        .datanodeUuid(DATANODE_UUID)
+        .conf(CONF)
+        .usageCheckFactory(MockSpaceUsageCheckFactory.NONE);
+    versionFile = StorageVolumeUtil.getVersionFile(rootDir);
+  }
+
+  @Test
+  public void testInitializeEmptyDbVolume() throws IOException {
+    DbVolume volume = volumeBuilder.build();
+
+    // The initial state of HddsVolume should be "NOT_FORMATTED" when
+    // clusterID is not specified and the version file should not be written
+    // to disk.
+    assertNull(volume.getClusterID());
+    assertEquals(StorageType.DEFAULT, volume.getStorageType());
+    assertEquals(HddsVolume.VolumeState.NOT_FORMATTED,
+        volume.getStorageState());
+    assertFalse("Version file should not be created when clusterID is not " +
+        "known.", versionFile.exists());
+
+    // Format the volume with clusterID.
+    volume.format(CLUSTER_ID);
+
+    // The state of HddsVolume after formatting with clusterID should be
+    // NORMAL and the version file should exist.
+    assertTrue("Volume format should create Version file",
+        versionFile.exists());
+    assertEquals(CLUSTER_ID, volume.getClusterID());
+    assertEquals(HddsVolume.VolumeState.NORMAL, volume.getStorageState());
+    assertEquals(0, volume.getHddsVolumeIDs().size());
+  }
+
+  @Test
+  public void testInitializeNonEmptyDbVolume() throws IOException {
+    DbVolume volume = volumeBuilder.build();
+
+    // The initial state of HddsVolume should be "NOT_FORMATTED" when
+    // clusterID is not specified and the version file should not be written
+    // to disk.
+    assertNull(volume.getClusterID());
+    assertEquals(StorageType.DEFAULT, volume.getStorageType());
+    assertEquals(HddsVolume.VolumeState.NOT_FORMATTED,
+        volume.getStorageState());
+    assertFalse("Version file should not be created when clusterID is not " +
+        "known.", versionFile.exists());
+
+    // Format the volume with clusterID.
+    volume.format(CLUSTER_ID);
+    volume.createWorkingDir(CLUSTER_ID, null);
+
+    // The clusterIdDir should be created
+    File clusterIdDir = new File(volume.getStorageDir(), CLUSTER_ID);
+    assertTrue(clusterIdDir.exists());
+
+    // Create some subdirectories to mock db instances under this volume.
+    int numSubDirs = 5;
+    File[] subdirs = new File[numSubDirs];
+    for (int i = 0; i < numSubDirs; i++) {
+      subdirs[i] = new File(clusterIdDir, UUID.randomUUID().toString());
+      boolean res = subdirs[i].mkdir();
+      assertTrue(res);
+    }
+
+    // Rebuild the same volume to simulate DN restart.
+    volume = volumeBuilder.build();
+    assertEquals(numSubDirs, volume.getHddsVolumeIDs().size());
+  }
+
+  @Test
+  public void testDbStoreClosedOnBadDbVolume() throws IOException {
+    ContainerTestUtils.enableSchemaV3(CONF);
+
+    DbVolume dbVolume = volumeBuilder.build();
+    dbVolume.format(CLUSTER_ID);
+    dbVolume.createWorkingDir(CLUSTER_ID, null);
+
+    MutableVolumeSet dbVolumeSet = mock(MutableVolumeSet.class);
+    when(dbVolumeSet.getVolumesList())
+        .thenReturn(Collections.singletonList(dbVolume));
+
+    MutableVolumeSet hddsVolumeSet = createHddsVolumeSet(3);
+    for (HddsVolume hddsVolume : StorageVolumeUtil.getHddsVolumesList(
+        hddsVolumeSet.getVolumesList())) {
+      hddsVolume.format(CLUSTER_ID);
+      hddsVolume.createWorkingDir(CLUSTER_ID, dbVolumeSet);
+    }
+
+    // The db handlers should be in the cache
+    assertEquals(3, DatanodeStoreCache.getInstance().size());
+
+    // Make the dbVolume a bad volume
+    dbVolume.failVolume();
+
+    // The db handlers should be removed from the cache
+    assertEquals(0, DatanodeStoreCache.getInstance().size());
+  }
+
+  private MutableVolumeSet createHddsVolumeSet(int volumeNum)
+      throws IOException {
+    File[] hddsVolumeDirs = new File[volumeNum];
+    StringBuilder hddsDirs = new StringBuilder();
+    for (int i = 0; i < volumeNum; i++) {
+      hddsVolumeDirs[i] = folder.newFolder();
+      hddsDirs.append(hddsVolumeDirs[i]).append(",");
+    }
+    CONF.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, hddsDirs.toString());
+    MutableVolumeSet hddsVolumeSet = new MutableVolumeSet(DATANODE_UUID,
+        CLUSTER_ID, CONF, null, StorageVolume.VolumeType.DATA_VOLUME, null);
+    return hddsVolumeSet;
+  }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java
index 3f664b4..9f26a0b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java
@@ -20,7 +20,6 @@
 import java.io.File;
 import java.io.IOException;
 import java.time.Duration;
-import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -32,15 +31,20 @@
 import org.apache.hadoop.hdds.fs.SpaceUsagePersistence;
 import org.apache.hadoop.hdds.fs.SpaceUsageSource;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
-import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 
 import static org.apache.hadoop.hdds.fs.MockSpaceUsagePersistence.inMemory;
 import static org.apache.hadoop.hdds.fs.MockSpaceUsageSource.fixed;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
+import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -71,7 +75,7 @@
         .datanodeUuid(DATANODE_UUID)
         .conf(CONF)
         .usageCheckFactory(MockSpaceUsageCheckFactory.NONE);
-    versionFile = HddsVolumeUtil.getVersionFile(rootDir);
+    versionFile = StorageVolumeUtil.getVersionFile(rootDir);
   }
 
   @Test
@@ -101,31 +105,6 @@
   }
 
   @Test
-  public void testReadPropertiesFromVersionFile() throws Exception {
-    HddsVolume volume = volumeBuilder.build();
-
-    volume.format(CLUSTER_ID);
-
-    Properties properties = DatanodeVersionFile.readFrom(versionFile);
-
-    String storageID = HddsVolumeUtil.getStorageID(properties, versionFile);
-    String clusterID = HddsVolumeUtil.getClusterID(
-        properties, versionFile, CLUSTER_ID);
-    String datanodeUuid = HddsVolumeUtil.getDatanodeUUID(
-        properties, versionFile, DATANODE_UUID);
-    long cTime = HddsVolumeUtil.getCreationTime(
-        properties, versionFile);
-    int layoutVersion = HddsVolumeUtil.getLayOutVersion(
-        properties, versionFile);
-
-    assertEquals(volume.getStorageID(), storageID);
-    assertEquals(volume.getClusterID(), clusterID);
-    assertEquals(volume.getDatanodeUuid(), datanodeUuid);
-    assertEquals(volume.getCTime(), cTime);
-    assertEquals(volume.getLayoutVersion(), layoutVersion);
-  }
-
-  @Test
   public void testShutdown() throws Exception {
     long initialUsedSpace = 250;
     AtomicLong savedUsedSpace = new AtomicLong(initialUsedSpace);
@@ -276,4 +255,131 @@
     // Shutdown the volume.
     volume.shutdown();
   }
+
+  @Test
+  public void testDbStoreCreatedWithoutDbVolumes() throws IOException {
+    ContainerTestUtils.enableSchemaV3(CONF);
+
+    HddsVolume volume = volumeBuilder.build();
+    volume.format(CLUSTER_ID);
+    volume.createWorkingDir(CLUSTER_ID, null);
+
+    // No DbVolume chosen and use the HddsVolume itself to hold
+    // a db instance.
+    assertNull(volume.getDbVolume());
+    File storageIdDir = new File(new File(volume.getStorageDir(),
+        CLUSTER_ID), volume.getStorageID());
+    assertEquals(volume.getDbParentDir(), storageIdDir);
+
+    // The db directory should exist
+    File containerDBFile = new File(volume.getDbParentDir(),
+        CONTAINER_DB_NAME);
+    assertTrue(containerDBFile.exists());
+
+    volume.shutdown();
+  }
+
+  @Test
+  public void testDbStoreCreatedWithDbVolumes() throws IOException {
+    ContainerTestUtils.enableSchemaV3(CONF);
+
+    // create the DbVolumeSet
+    MutableVolumeSet dbVolumeSet = createDbVolumeSet();
+
+    HddsVolume volume = volumeBuilder.build();
+    volume.format(CLUSTER_ID);
+    volume.createWorkingDir(CLUSTER_ID, dbVolumeSet);
+
+    // DbVolume chosen.
+    assertNotNull(volume.getDbVolume());
+
+    File storageIdDir = new File(new File(volume.getDbVolume()
+        .getStorageDir(), CLUSTER_ID), volume.getStorageID());
+    // Db parent dir should be set to a subdir under the dbVolume.
+    assertEquals(volume.getDbParentDir(), storageIdDir);
+
+    // The db directory should exist
+    File containerDBFile = new File(volume.getDbParentDir(),
+        CONTAINER_DB_NAME);
+    assertTrue(containerDBFile.exists());
+
+    volume.shutdown();
+  }
+
+  @Test
+  public void testDbStoreClosedOnBadVolumeWithoutDbVolumes()
+      throws IOException {
+    ContainerTestUtils.enableSchemaV3(CONF);
+
+    HddsVolume volume = volumeBuilder.build();
+    volume.format(CLUSTER_ID);
+    volume.createWorkingDir(CLUSTER_ID, null);
+
+    // No DbVolume chosen and use the HddsVolume itself to hold
+    // a db instance.
+    assertNull(volume.getDbVolume());
+    File storageIdDir = new File(new File(volume.getStorageDir(),
+        CLUSTER_ID), volume.getStorageID());
+    assertEquals(volume.getDbParentDir(), storageIdDir);
+
+    // The db directory should exist
+    File containerDBFile = new File(volume.getDbParentDir(),
+        CONTAINER_DB_NAME);
+    assertTrue(containerDBFile.exists());
+    assertNotNull(DatanodeStoreCache.getInstance().getDB(
+        containerDBFile.getAbsolutePath()));
+
+    // Make it a bad volume
+    volume.failVolume();
+
+    // The db should be removed from cache
+    assertNull(DatanodeStoreCache.getInstance().getDB(
+        containerDBFile.getAbsolutePath()));
+  }
+
+  @Test
+  public void testDbStoreClosedOnBadVolumeWithDbVolumes() throws IOException {
+    ContainerTestUtils.enableSchemaV3(CONF);
+
+    // create the DbVolumeSet
+    MutableVolumeSet dbVolumeSet = createDbVolumeSet();
+
+    HddsVolume volume = volumeBuilder.build();
+    volume.format(CLUSTER_ID);
+    volume.createWorkingDir(CLUSTER_ID, dbVolumeSet);
+
+    // DbVolume chosen.
+    assertNotNull(volume.getDbVolume());
+
+    File storageIdDir = new File(new File(volume.getDbVolume()
+        .getStorageDir(), CLUSTER_ID), volume.getStorageID());
+    // Db parent dir should be set to a subdir under the dbVolume.
+    assertEquals(volume.getDbParentDir(), storageIdDir);
+
+    // The db directory should exist
+    File containerDBFile = new File(volume.getDbParentDir(),
+        CONTAINER_DB_NAME);
+    assertTrue(containerDBFile.exists());
+    assertNotNull(DatanodeStoreCache.getInstance().getDB(
+        containerDBFile.getAbsolutePath()));
+
+    // Make it a bad volume
+    volume.failVolume();
+
+    // The db should be removed from cache
+    assertNull(DatanodeStoreCache.getInstance().getDB(
+        containerDBFile.getAbsolutePath()));
+  }
+
+  private MutableVolumeSet createDbVolumeSet() throws IOException {
+    File dbVolumeDir = folder.newFolder();
+    CONF.set(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR,
+        dbVolumeDir.getAbsolutePath());
+    MutableVolumeSet dbVolumeSet = new MutableVolumeSet(DATANODE_UUID,
+        CLUSTER_ID, CONF, null, StorageVolume.VolumeType.DB_VOLUME,
+        null);
+    dbVolumeSet.getVolumesList().get(0).format(CLUSTER_ID);
+    dbVolumeSet.getVolumesList().get(0).createWorkingDir(CLUSTER_ID, null);
+    return dbVolumeSet;
+  }
 }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestStorageVolume.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestStorageVolume.java
new file mode 100644
index 0000000..5f01520
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestStorageVolume.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.fs.MockSpaceUsageCheckFactory;
+import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for StorageVolume.
+ */
+public class TestStorageVolume {
+
+  private static final String DATANODE_UUID = UUID.randomUUID().toString();
+  private static final String CLUSTER_ID = UUID.randomUUID().toString();
+  private static final OzoneConfiguration CONF = new OzoneConfiguration();
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  private HddsVolume.Builder volumeBuilder;
+  private File versionFile;
+
+  @Before
+  public void setup() throws Exception {
+    File rootDir = new File(folder.getRoot(), HddsVolume.HDDS_VOLUME_DIR);
+    volumeBuilder = new HddsVolume.Builder(folder.getRoot().getPath())
+        .datanodeUuid(DATANODE_UUID)
+        .conf(CONF)
+        .usageCheckFactory(MockSpaceUsageCheckFactory.NONE);
+    versionFile = StorageVolumeUtil.getVersionFile(rootDir);
+  }
+
+  @Test
+  public void testReadPropertiesFromVersionFile() throws Exception {
+    HddsVolume volume = volumeBuilder.build();
+
+    volume.format(CLUSTER_ID);
+
+    Properties properties = DatanodeVersionFile.readFrom(versionFile);
+
+    String storageID = StorageVolumeUtil.getStorageID(properties, versionFile);
+    String clusterID = StorageVolumeUtil.getClusterID(
+        properties, versionFile, CLUSTER_ID);
+    String datanodeUuid = StorageVolumeUtil.getDatanodeUUID(
+        properties, versionFile, DATANODE_UUID);
+    long cTime = StorageVolumeUtil.getCreationTime(
+        properties, versionFile);
+    int layoutVersion = StorageVolumeUtil.getLayOutVersion(
+        properties, versionFile);
+
+    assertEquals(volume.getStorageID(), storageID);
+    assertEquals(volume.getClusterID(), clusterID);
+    assertEquals(volume.getDatanodeUuid(), datanodeUuid);
+    assertEquals(volume.getCTime(), cTime);
+    assertEquals(volume.getLayoutVersion(), layoutVersion);
+  }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSetDiskChecks.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSetDiskChecks.java
index 76e771d..84263de 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSetDiskChecks.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSetDiskChecks.java
@@ -30,6 +30,7 @@
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
 import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
 import org.apache.ozone.test.GenericTestUtils;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -117,6 +118,7 @@
     final int numBadVolumes = 2;
 
     conf = getConfWithDataNodeDirs(numVolumes);
+    ContainerTestUtils.enableSchemaV3(conf);
     StorageVolumeChecker dummyChecker =
         new DummyChecker(conf, new Timer(), numBadVolumes);
     final MutableVolumeSet volumeSet = new MutableVolumeSet(
@@ -127,6 +129,10 @@
         UUID.randomUUID().toString(), conf, null,
         StorageVolume.VolumeType.META_VOLUME,
         dummyChecker);
+    final MutableVolumeSet dbVolumeSet = new MutableVolumeSet(
+        UUID.randomUUID().toString(), conf, null,
+        StorageVolume.VolumeType.DB_VOLUME,
+        dummyChecker);
 
     Assert.assertEquals(volumeSet.getFailedVolumesList().size(),
         numBadVolumes);
@@ -136,8 +142,14 @@
         numBadVolumes);
     Assert.assertEquals(metaVolumeSet.getVolumesList().size(),
         numVolumes - numBadVolumes);
+    Assert.assertEquals(dbVolumeSet.getFailedVolumesList().size(),
+        numBadVolumes);
+    Assert.assertEquals(dbVolumeSet.getVolumesList().size(),
+        numVolumes - numBadVolumes);
+
     volumeSet.shutdown();
     metaVolumeSet.shutdown();
+    dbVolumeSet.shutdown();
   }
 
   /**
@@ -148,6 +160,7 @@
     final int numVolumes = 5;
 
     conf = getConfWithDataNodeDirs(numVolumes);
+    ContainerTestUtils.enableSchemaV3(conf);
     StorageVolumeChecker dummyChecker =
         new DummyChecker(conf, new Timer(), numVolumes);
 
@@ -159,13 +172,21 @@
         UUID.randomUUID().toString(), conf, null,
         StorageVolume.VolumeType.META_VOLUME,
         dummyChecker);
+    final MutableVolumeSet dbVolumeSet = new MutableVolumeSet(
+        UUID.randomUUID().toString(), conf, null,
+        StorageVolume.VolumeType.DB_VOLUME,
+        dummyChecker);
 
     assertEquals(volumeSet.getFailedVolumesList().size(), numVolumes);
     assertEquals(volumeSet.getVolumesList().size(), 0);
     assertEquals(metaVolumeSet.getFailedVolumesList().size(), numVolumes);
     assertEquals(metaVolumeSet.getVolumesList().size(), 0);
+    assertEquals(dbVolumeSet.getFailedVolumesList().size(), numVolumes);
+    assertEquals(dbVolumeSet.getVolumesList().size(), 0);
+
     volumeSet.shutdown();
     metaVolumeSet.shutdown();
+    dbVolumeSet.shutdown();
   }
 
   /**
@@ -188,10 +209,19 @@
     }
     ozoneConf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR,
         String.join(",", metaDirs));
+
+    final List<String> dbDirs = new ArrayList<>();
+    for (int i = 0; i < numDirs; ++i) {
+      dbDirs.add(GenericTestUtils.getRandomizedTestDir().getPath());
+    }
+    ozoneConf.set(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR,
+        String.join(",", dbDirs));
+
     DatanodeConfiguration dnConf =
         ozoneConf.getObject(DatanodeConfiguration.class);
     dnConf.setFailedDataVolumesTolerated(numDirs * 2);
     dnConf.setFailedMetadataVolumesTolerated(numDirs * 2);
+    dnConf.setFailedDbVolumesTolerated(numDirs * 2);
     ozoneConf.setFromObject(dnConf);
     return ozoneConf;
   }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/ContainerTestVersionInfo.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/ContainerTestVersionInfo.java
new file mode 100644
index 0000000..c1292ea
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/ContainerTestVersionInfo.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_VERSIONS;
+
+/**
+ * Class to hold version info for container data and metadata.
+ * - SchemaVersion: metadata schema version
+ * - ChunkLayOutVersion: data layout version
+ */
+public class ContainerTestVersionInfo {
+  private final String schemaVersion;
+  private final ContainerLayoutVersion layout;
+
+  public ContainerTestVersionInfo(String schemaVersion,
+      ContainerLayoutVersion layout) {
+    this.schemaVersion = schemaVersion;
+    this.layout = layout;
+  }
+
+  private static List<ContainerTestVersionInfo> layoutList = new ArrayList<>();
+  static {
+    for (ContainerLayoutVersion ch : ContainerLayoutVersion.getAllVersions()) {
+      for (String sch : SCHEMA_VERSIONS) {
+        layoutList.add(new ContainerTestVersionInfo(sch, ch));
+      }
+    }
+  }
+
+  public String getSchemaVersion() {
+    return this.schemaVersion;
+  }
+
+  public ContainerLayoutVersion getLayout() {
+    return this.layout;
+  }
+
+  public static Iterable<Object[]> versionParameters() {
+    return layoutList.stream().map(each -> new Object[] {each})
+        .collect(toList());
+  }
+
+  public static List<ContainerTestVersionInfo> getLayoutList() {
+    return layoutList;
+  }
+  public static void setTestSchemaVersion(String schemaVersion,
+      OzoneConfiguration conf) {
+    if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      ContainerTestUtils.enableSchemaV3(conf);
+    } else {
+      ContainerTestUtils.disableSchemaV3(conf);
+    }
+  }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java
index 7f14ccf..3734c5b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java
@@ -20,8 +20,6 @@
 
 import java.io.File;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -42,18 +40,20 @@
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
 import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.ozone.test.GenericTestUtils;
 
+import static java.util.stream.Collectors.toList;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
-import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_BLOCK;
-import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_CHUNK;
 
 import org.junit.After;
+
+import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.createDbInstancesForTestIfNeeded;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -75,29 +75,47 @@
   private MutableVolumeSet volumeSet;
   private OzoneConfiguration conf;
   private File testRoot;
-  private ReferenceCountedDB db;
+  private DBHandle db;
   private final ContainerLayoutVersion layout;
+  private String schemaVersion;
+  private String datanodeID = UUID.randomUUID().toString();
+  private String clusterID = UUID.randomUUID().toString();
 
-  public TestKeyValueBlockIterator(ContainerLayoutVersion layout) {
-    this.layout = layout;
+  public TestKeyValueBlockIterator(ContainerTestVersionInfo versionInfo,
+      String keySeparator) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    this.conf = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, conf);
+    DatanodeConfiguration dc = conf.getObject(DatanodeConfiguration.class);
+    dc.setContainerSchemaV3KeySeparator(keySeparator);
+    conf.setFromObject(dc);
   }
 
   @Parameterized.Parameters
-  public static Collection<Object[]> data() {
-    return Arrays.asList(new Object[][] {
-        {FILE_PER_CHUNK},
-        {FILE_PER_BLOCK}
-    });
+  public static Iterable<Object[]> data() {
+    List listA =
+        ContainerTestVersionInfo.getLayoutList().stream().map(
+            each -> new Object[] {each, ""})
+            .collect(toList());
+    List listB =
+        ContainerTestVersionInfo.getLayoutList().stream().map(
+            each -> new Object[] {each,
+                new DatanodeConfiguration().getContainerSchemaV3KeySeparator()})
+            .collect(toList());
+
+    listB.addAll(listA);
+    return listB;
   }
 
   @Before
   public void setUp() throws Exception {
     testRoot = GenericTestUtils.getRandomizedTestDir();
-    conf = new OzoneConfiguration();
     conf.set(HDDS_DATANODE_DIR_KEY, testRoot.getAbsolutePath());
     conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testRoot.getAbsolutePath());
-    volumeSet = new MutableVolumeSet(UUID.randomUUID().toString(), conf, null,
+    volumeSet = new MutableVolumeSet(datanodeID, clusterID, conf, null,
         StorageVolume.VolumeType.DATA_VOLUME, null);
+    createDbInstancesForTestIfNeeded(volumeSet, clusterID, clusterID, conf);
 
     containerData = new KeyValueContainerData(105L,
             layout,
@@ -105,8 +123,8 @@
             UUID.randomUUID().toString());
     // Init the container.
     container = new KeyValueContainer(containerData, conf);
-    container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), UUID
-            .randomUUID().toString());
+    container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
+        clusterID);
     db = BlockUtils.getDB(containerData, conf);
   }
 
@@ -115,6 +133,7 @@
   public void tearDown() throws Exception {
     db.close();
     db.cleanup();
+    BlockUtils.shutdownCache(conf);
     volumeSet.shutdown();
     FileUtil.fullyDelete(testRoot);
   }
@@ -129,7 +148,7 @@
     // Default filter used is all unprefixed blocks.
     List<Long> unprefixedBlockIDs = blockIDs.get("");
     try (BlockIterator<BlockData> keyValueBlockIterator =
-                db.getStore().getBlockIterator()) {
+                db.getStore().getBlockIterator(CONTAINER_ID)) {
 
       Iterator<Long> blockIDIter = unprefixedBlockIDs.iterator();
       while (keyValueBlockIterator.hasNext()) {
@@ -161,7 +180,7 @@
   public void testKeyValueBlockIteratorWithNextBlock() throws Exception {
     List<Long> blockIDs = createContainerWithBlocks(CONTAINER_ID, 2);
     try (BlockIterator<BlockData> keyValueBlockIterator =
-                db.getStore().getBlockIterator()) {
+                db.getStore().getBlockIterator(CONTAINER_ID)) {
       assertEquals((long)blockIDs.get(0),
               keyValueBlockIterator.nextBlock().getLocalID());
       assertEquals((long)blockIDs.get(1),
@@ -180,7 +199,7 @@
   public void testKeyValueBlockIteratorWithHasNext() throws Exception {
     List<Long> blockIDs = createContainerWithBlocks(CONTAINER_ID, 2);
     try (BlockIterator<BlockData> blockIter =
-                db.getStore().getBlockIterator()) {
+                db.getStore().getBlockIterator(CONTAINER_ID)) {
 
       // Even calling multiple times hasNext() should not move entry forward.
       assertTrue(blockIter.hasNext());
@@ -218,8 +237,8 @@
     Map<String, List<Long>> blockIDs = createContainerWithBlocks(CONTAINER_ID,
             normalBlocks, deletingBlocks);
     try (BlockIterator<BlockData> keyValueBlockIterator =
-                db.getStore().getBlockIterator(
-                        MetadataKeyFilters.getDeletingKeyFilter())) {
+                db.getStore().getBlockIterator(CONTAINER_ID,
+                        containerData.getDeletingBlockKeyFilter())) {
       List<Long> deletingBlockIDs =
               blockIDs.get(OzoneConsts.DELETING_KEY_PREFIX);
       int counter = 0;
@@ -239,7 +258,7 @@
       Exception {
     createContainerWithBlocks(CONTAINER_ID, 0, 5);
     try (BlockIterator<BlockData> keyValueBlockIterator =
-                db.getStore().getBlockIterator()) {
+                db.getStore().getBlockIterator(CONTAINER_ID)) {
       //As all blocks are deleted blocks, blocks does not match with normal key
       // filter.
       assertFalse(keyValueBlockIterator.hasNext());
@@ -281,13 +300,14 @@
     Map<String, List<Long>> blockIDs = createContainerWithBlocks(CONTAINER_ID,
             prefixCounts);
     // Test deleting filter.
-    testWithFilter(MetadataKeyFilters.getDeletingKeyFilter(),
+    testWithFilter(containerData.getDeletingBlockKeyFilter(),
             blockIDs.get(OzoneConsts.DELETING_KEY_PREFIX));
 
     // Test arbitrary filter.
+    String schemaPrefix = containerData.containerPrefix();
     MetadataKeyFilters.KeyPrefixFilter secondFilter =
             new MetadataKeyFilters.KeyPrefixFilter()
-            .addFilter(secondPrefix);
+            .addFilter(schemaPrefix + secondPrefix);
     testWithFilter(secondFilter, blockIDs.get(secondPrefix));
   }
 
@@ -297,7 +317,7 @@
   private void testWithFilter(MetadataKeyFilters.KeyPrefixFilter filter,
                               List<Long> expectedIDs) throws Exception {
     try (BlockIterator<BlockData> iterator =
-                db.getStore().getBlockIterator(filter)) {
+                db.getStore().getBlockIterator(CONTAINER_ID, filter)) {
       // Test seek.
       iterator.seekToFirst();
       long firstID = iterator.nextBlock().getLocalID();
@@ -372,8 +392,7 @@
             Map<String, Integer> prefixCounts) throws Exception {
     // Create required block data.
     Map<String, List<Long>> blockIDs = new HashMap<>();
-    try (ReferenceCountedDB metadataStore = BlockUtils.getDB(containerData,
-        conf)) {
+    try (DBHandle metadataStore = BlockUtils.getDB(containerData, conf)) {
 
       List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
       ChunkInfo info = new ChunkInfo("chunkfile", 0, 1024);
@@ -384,6 +403,7 @@
       // prefix.
       Table<String, BlockData> blockDataTable =
               metadataStore.getStore().getBlockDataTable();
+      String schemaPrefix = containerData.containerPrefix();
 
       for (Map.Entry<String, Integer> entry: prefixCounts.entrySet()) {
         String prefix = entry.getKey();
@@ -395,8 +415,8 @@
           blockIndex++;
           BlockData blockData = new BlockData(blockID);
           blockData.setChunks(chunkList);
-          String localID = prefix + blockID.getLocalID();
-          blockDataTable.put(localID, blockData);
+          String blockKey = schemaPrefix + prefix + blockID.getLocalID();
+          blockDataTable.put(blockKey, blockData);
           blockIDs.get(prefix).add(blockID.getLocalID());
         }
       }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index 1a19459..0d344d0a 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -34,6 +34,7 @@
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.utils.db.DatanodeDBProfile;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
@@ -47,7 +48,6 @@
 import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
 import org.apache.ozone.test.GenericTestUtils;
 import org.apache.hadoop.util.DiskChecker;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 
 import org.junit.Assert;
 import org.junit.Before;
@@ -105,19 +105,22 @@
   private UUID datanodeId;
 
   private final ContainerLayoutVersion layout;
+  private String schemaVersion;
 
   // Use one configuration object across parameterized runs of tests.
   // This preserves the column family options in the container options
   // cache for testContainersShareColumnFamilyOptions.
   private static final OzoneConfiguration CONF = new OzoneConfiguration();
 
-  public TestKeyValueContainer(ContainerLayoutVersion layout) {
-    this.layout = layout;
+  public TestKeyValueContainer(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, CONF);
   }
 
   @Parameterized.Parameters
   public static Iterable<Object[]> parameters() {
-    return ContainerLayoutTestInfo.containerLayoutParameters();
+    return ContainerTestVersionInfo.versionParameters();
   }
 
   @Before
@@ -126,6 +129,7 @@
     HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
         .getAbsolutePath()).conf(CONF).datanodeUuid(datanodeId
         .toString()).build();
+    StorageVolumeUtil.checkVolume(hddsVolume, scmId, scmId, CONF, null, null);
 
     volumeSet = mock(MutableVolumeSet.class);
     volumeChoosingPolicy = mock(RoundRobinVolumeChoosingPolicy.class);
@@ -234,13 +238,13 @@
             keyValueContainerData.getLayoutVersion(),
             keyValueContainerData.getMaxSize(), UUID.randomUUID().toString(),
             datanodeId.toString());
+    containerData.setSchemaVersion(keyValueContainerData.getSchemaVersion());
     KeyValueContainer container = new KeyValueContainer(containerData, CONF);
 
     HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(
         StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()), 1);
-    String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
 
-    container.populatePathFields(scmId, containerVolume, hddsVolumeDir);
+    container.populatePathFields(scmId, containerVolume);
     try (FileInputStream fis = new FileInputStream(folderToExport)) {
       container.importContainerData(fis, packer);
     }
@@ -276,12 +280,12 @@
             keyValueContainerData.getLayoutVersion(),
             keyValueContainerData.getMaxSize(), UUID.randomUUID().toString(),
             datanodeId.toString());
+    containerData.setSchemaVersion(keyValueContainerData.getSchemaVersion());
     container = new KeyValueContainer(containerData, CONF);
 
     containerVolume = volumeChoosingPolicy.chooseVolume(
         StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()), 1);
-    hddsVolumeDir = containerVolume.getHddsRootDir().toString();
-    container.populatePathFields(scmId, containerVolume, hddsVolumeDir);
+    container.populatePathFields(scmId, containerVolume);
     try {
       FileInputStream fis = new FileInputStream(folderToExport);
       fis.close();
@@ -320,19 +324,20 @@
    * Add some keys to the container.
    */
   private void populate(long numberOfKeysToWrite) throws IOException {
-    try (ReferenceCountedDB metadataStore =
-        BlockUtils.getDB(keyValueContainer.getContainerData(), CONF)) {
+    KeyValueContainerData cData = keyValueContainer.getContainerData();
+    try (DBHandle metadataStore = BlockUtils.getDB(cData, CONF)) {
       Table<String, BlockData> blockDataTable =
               metadataStore.getStore().getBlockDataTable();
 
       for (long i = 0; i < numberOfKeysToWrite; i++) {
-        blockDataTable.put("test" + i, new BlockData(new BlockID(i, i)));
+        blockDataTable.put(cData.blockKey(i),
+            new BlockData(new BlockID(i, i)));
       }
 
       // As now when we put blocks, we increment block count and update in DB.
       // As for test, we are doing manually so adding key count to DB.
       metadataStore.getStore().getMetadataTable()
-              .put(OzoneConsts.BLOCK_COUNT, numberOfKeysToWrite);
+              .put(cData.blockCountKey(), numberOfKeysToWrite);
     }
 
     Map<String, String> metadata = new HashMap<>();
@@ -425,8 +430,13 @@
 
     assertFalse("Container File still exists",
         keyValueContainer.getContainerFile().exists());
-    assertFalse("Container DB file still exists",
-        keyValueContainer.getContainerDBFile().exists());
+
+    if (schemaVersion.equals(OzoneConsts.SCHEMA_V3)) {
+      assertTrue(keyValueContainer.getContainerDBFile().exists());
+    } else {
+      assertFalse("Container DB file still exists",
+          keyValueContainer.getContainerDBFile().exists());
+    }
   }
 
   @Test
@@ -503,14 +513,13 @@
   }
 
   @Test
-  public void testContainerRocksDB() throws Exception {
+  public void testContainerRocksDB() throws IOException {
     closeContainer();
     keyValueContainer = new KeyValueContainer(
         keyValueContainerData, CONF);
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
 
-    try (ReferenceCountedDB db =
-        BlockUtils.getDB(keyValueContainerData, CONF)) {
+    try (DBHandle db = BlockUtils.getDB(keyValueContainerData, CONF)) {
       RDBStore store = (RDBStore) db.getStore().getStore();
       long defaultCacheSize = 64 * OzoneConsts.MB;
       long cacheSize = Long.parseLong(store
@@ -563,7 +572,7 @@
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
 
     DatanodeDBProfile outProfile1;
-    try (ReferenceCountedDB db1 =
+    try (DBHandle db1 =
         BlockUtils.getDB(keyValueContainer.getContainerData(), CONF)) {
       DatanodeStore store1 = db1.getStore();
       Assert.assertTrue(store1 instanceof AbstractDatanodeStore);
@@ -583,7 +592,7 @@
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
 
     DatanodeDBProfile outProfile2;
-    try (ReferenceCountedDB db2 =
+    try (DBHandle db2 =
         BlockUtils.getDB(keyValueContainer.getContainerData(), otherConf)) {
       DatanodeStore store2 = db2.getStore();
       Assert.assertTrue(store2 instanceof AbstractDatanodeStore);
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerCheck.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerCheck.java
index 99812f3..b5694d7 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerCheck.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerCheck.java
@@ -25,10 +25,10 @@
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.ozoneimpl.ContainerScrubberConfiguration;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -47,9 +47,8 @@
 public class TestKeyValueContainerCheck
     extends TestKeyValueContainerIntegrityChecks {
 
-  public TestKeyValueContainerCheck(ContainerLayoutTestInfo
-      containerLayoutTestInfo) {
-    super(containerLayoutTestInfo);
+  public TestKeyValueContainerCheck(ContainerTestVersionInfo versionInfo) {
+    super(versionInfo);
   }
 
   /**
@@ -71,7 +70,7 @@
 
     KeyValueContainerCheck kvCheck =
         new KeyValueContainerCheck(containerData.getMetadataPath(), conf,
-            containerID);
+            containerID, containerData.getVolume());
 
     // first run checks on a Open Container
     boolean valid = kvCheck.fastCheck();
@@ -106,16 +105,14 @@
 
     KeyValueContainerCheck kvCheck =
         new KeyValueContainerCheck(containerData.getMetadataPath(), conf,
-            containerID);
+            containerID, containerData.getVolume());
 
-    File metaDir = new File(containerData.getMetadataPath());
     File dbFile = KeyValueContainerLocationUtil
-        .getContainerDBFile(metaDir, containerID);
+        .getContainerDBFile(containerData);
     containerData.setDbFile(dbFile);
-    try (ReferenceCountedDB ignored =
-            BlockUtils.getDB(containerData, conf);
+    try (DBHandle ignored = BlockUtils.getDB(containerData, conf);
         BlockIterator<BlockData> kvIter =
-                ignored.getStore().getBlockIterator()) {
+                ignored.getStore().getBlockIterator(containerID)) {
       BlockData block = kvIter.nextBlock();
       assertFalse(block.getChunks().isEmpty());
       ContainerProtos.ChunkInfo c = block.getChunks().get(0);
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerIntegrityChecks.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerIntegrityChecks.java
index 63bf5d6..51e7283 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerIntegrityChecks.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerIntegrityChecks.java
@@ -23,12 +23,12 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.Checksum;
 import org.apache.hadoop.ozone.common.ChecksumData;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
@@ -36,7 +36,6 @@
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
 import org.apache.ozone.test.GenericTestUtils;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.runners.Parameterized;
@@ -44,15 +43,14 @@
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.util.Arrays;
 import java.util.ArrayList;
 import java.nio.ByteBuffer;
-import java.util.Collection;
 import java.util.List;
 import java.util.UUID;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
+import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.createDbInstancesForTestIfNeeded;
 import static org.junit.Assert.assertNotNull;
 
 /**
@@ -68,36 +66,44 @@
   private OzoneConfiguration conf;
   private File testRoot;
   private ChunkManager chunkManager;
+  private String datanodeID = UUID.randomUUID().toString();
+  private String clusterID = UUID.randomUUID().toString();
 
   protected static final int UNIT_LEN = 1024;
   protected static final int CHUNK_LEN = 3 * UNIT_LEN;
   protected static final int CHUNKS_PER_BLOCK = 4;
 
-  public TestKeyValueContainerIntegrityChecks(ContainerLayoutTestInfo
-      containerLayoutTestInfo) {
-    this.containerLayoutTestInfo = containerLayoutTestInfo;
+  public TestKeyValueContainerIntegrityChecks(
+      ContainerTestVersionInfo versionInfo) {
+    this.conf = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(
+        versionInfo.getSchemaVersion(), conf);
+    if (versionInfo.getLayout()
+        .equals(ContainerLayoutVersion.FILE_PER_BLOCK)) {
+      containerLayoutTestInfo = ContainerLayoutTestInfo.FILE_PER_BLOCK;
+    } else {
+      containerLayoutTestInfo = ContainerLayoutTestInfo.FILE_PER_CHUNK;
+    }
   }
 
-  @Parameterized.Parameters public static Collection<Object[]> data() {
-    return Arrays.asList(new Object[][] {
-        {ContainerLayoutTestInfo.FILE_PER_CHUNK},
-        {ContainerLayoutTestInfo.FILE_PER_BLOCK}
-    });
+  @Parameterized.Parameters public static Iterable<Object[]> data() {
+    return ContainerTestVersionInfo.versionParameters();
   }
 
   @Before public void setUp() throws Exception {
     LOG.info("Testing  layout:{}", containerLayoutTestInfo.getLayout());
     this.testRoot = GenericTestUtils.getRandomizedTestDir();
-    conf = new OzoneConfiguration();
     conf.set(HDDS_DATANODE_DIR_KEY, testRoot.getAbsolutePath());
     conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testRoot.getAbsolutePath());
     containerLayoutTestInfo.updateConfig(conf);
-    volumeSet = new MutableVolumeSet(UUID.randomUUID().toString(), conf, null,
-        StorageVolume.VolumeType.DATA_VOLUME, null);
+    volumeSet = new MutableVolumeSet(UUID.randomUUID().toString(), clusterID,
+        conf, null, StorageVolume.VolumeType.DATA_VOLUME, null);
+    createDbInstancesForTestIfNeeded(volumeSet, clusterID, clusterID, conf);
     chunkManager = containerLayoutTestInfo.createChunkManager(true, null);
   }
 
   @After public void teardown() {
+    BlockUtils.shutdownCache(conf);
     volumeSet.shutdown();
     FileUtil.fullyDelete(testRoot);
   }
@@ -139,8 +145,8 @@
         UUID.randomUUID().toString(), UUID.randomUUID().toString());
     KeyValueContainer container = new KeyValueContainer(containerData, conf);
     container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
-        UUID.randomUUID().toString());
-    try (ReferenceCountedDB metadataStore = BlockUtils.getDB(containerData,
+        clusterID);
+    try (DBHandle metadataStore = BlockUtils.getDB(containerData,
         conf)) {
       assertNotNull(containerData.getChunksPath());
       File chunksPath = new File(containerData.getChunksPath());
@@ -166,10 +172,10 @@
         blockData.setChunks(chunkList);
 
         // normal key
-        String key = Long.toString(blockID.getLocalID());
+        String key = containerData.blockKey(blockID.getLocalID());
         if (i >= normalBlocks) {
           // deleted key
-          key = OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID();
+          key = containerData.deletingBlockKey(blockID.getLocalID());
         }
         metadataStore.getStore().getBlockDataTable().put(key, blockData);
       }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerMetadataInspector.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerMetadataInspector.java
index 9656c28..8195e6f 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerMetadataInspector.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerMetadataInspector.java
@@ -25,8 +25,8 @@
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerInspector;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.utils.ContainerInspectorUtil;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.apache.log4j.PatternLayout;
@@ -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);
   }
 
   @Test
@@ -290,23 +290,23 @@
 
   public void setDBBlockAndByteCounts(KeyValueContainerData containerData,
       long blockCount, long byteCount) throws Exception {
-    try (ReferenceCountedDB db = BlockUtils.getDB(containerData, getConf())) {
+    try (DBHandle db = BlockUtils.getDB(containerData, getConf())) {
       Table<String, Long> metadataTable = db.getStore().getMetadataTable();
       // Don't care about in memory state. Just change the DB values.
-      metadataTable.put(OzoneConsts.BLOCK_COUNT, blockCount);
-      metadataTable.put(OzoneConsts.CONTAINER_BYTES_USED, byteCount);
+      metadataTable.put(containerData.blockCountKey(), blockCount);
+      metadataTable.put(containerData.bytesUsedKey(), byteCount);
     }
   }
 
   public void checkDBBlockAndByteCounts(KeyValueContainerData containerData,
       long expectedBlockCount, long expectedBytesUsed) throws Exception {
-    try (ReferenceCountedDB db = BlockUtils.getDB(containerData, getConf())) {
+    try (DBHandle db = BlockUtils.getDB(containerData, getConf())) {
       Table<String, Long> metadataTable = db.getStore().getMetadataTable();
 
-      long bytesUsed = metadataTable.get(OzoneConsts.CONTAINER_BYTES_USED);
+      long bytesUsed = metadataTable.get(containerData.bytesUsedKey());
       Assert.assertEquals(expectedBytesUsed, bytesUsed);
 
-      long blockCount = metadataTable.get(OzoneConsts.BLOCK_COUNT);
+      long blockCount = metadataTable.get(containerData.blockCountKey());
       Assert.assertEquals(expectedBlockCount, blockCount);
     }
   }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
index 793aea5..c15841b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
@@ -88,14 +88,19 @@
   private static final AtomicInteger CONTAINER_ID = new AtomicInteger(1);
 
   private final ContainerLayoutVersion layout;
+  private final String schemaVersion;
+  private OzoneConfiguration conf;
 
-  public TestTarContainerPacker(ContainerLayoutVersion layout) {
-    this.layout = layout;
+  public TestTarContainerPacker(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    this.conf = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, conf);
   }
 
   @Parameterized.Parameters
   public static Iterable<Object[]> parameters() {
-    return ContainerLayoutTestInfo.containerLayoutParameters();
+    return ContainerTestVersionInfo.versionParameters();
   }
 
   @BeforeClass
@@ -125,14 +130,17 @@
     Path containerDir = dir.resolve("container" + id);
     Path dbDir = containerDir.resolve("db");
     Path dataDir = containerDir.resolve("data");
+    Path metaDir = containerDir.resolve("metadata");
+    Files.createDirectories(metaDir);
     Files.createDirectories(dbDir);
     Files.createDirectories(dataDir);
 
     KeyValueContainerData containerData = new KeyValueContainerData(
         id, layout,
         -1, UUID.randomUUID().toString(), UUID.randomUUID().toString());
+    containerData.setSchemaVersion(schemaVersion);
     containerData.setChunksPath(dataDir.toString());
-    containerData.setMetadataPath(dbDir.getParent().toString());
+    containerData.setMetadataPath(metaDir.toString());
     containerData.setDbFile(dbDir.toFile());
 
     return containerData;
@@ -140,10 +148,7 @@
 
   @Test
   public void pack() throws IOException, CompressorException {
-
     //GIVEN
-    OzoneConfiguration conf = new OzoneConfiguration();
-
     KeyValueContainerData sourceContainerData =
         createContainer(SOURCE_CONTAINER_ROOT);
 
@@ -211,7 +216,7 @@
     }
 
     assertExampleMetadataDbIsGood(
-        destinationContainerData.getDbFile().toPath(),
+        TarContainerPacker.getDbPath(destinationContainerData),
         TEST_DB_FILE_NAME);
     assertExampleChunkFileIsGood(
         Paths.get(destinationContainerData.getChunksPath()),
@@ -240,7 +245,8 @@
     KeyValueContainerData dest = unpackContainerData(containerFile);
 
     // THEN
-    assertExampleMetadataDbIsGood(dest.getDbFile().toPath(), fileName);
+    assertExampleMetadataDbIsGood(
+        TarContainerPacker.getDbPath(dest), fileName);
   }
 
   @Test
@@ -300,7 +306,6 @@
   private KeyValueContainerData unpackContainerData(File containerFile)
       throws IOException {
     try (FileInputStream input = new FileInputStream(containerFile)) {
-      OzoneConfiguration conf = new OzoneConfiguration();
       KeyValueContainerData data = createContainer(DEST_CONTAINER_ROOT);
       KeyValueContainer container = new KeyValueContainer(data, conf);
       packer.unpackContainerData(container, input);
@@ -320,29 +325,26 @@
   private File writeChunkFile(
       KeyValueContainerData containerData, String chunkFileName)
       throws IOException {
-    Path path = Paths.get(containerData.getChunksPath())
-        .resolve(chunkFileName);
-    Files.createDirectories(path.getParent());
-    File file = path.toFile();
-    FileOutputStream fileStream = new FileOutputStream(file);
-    try (OutputStreamWriter writer = new OutputStreamWriter(fileStream,
-        UTF_8)) {
-      IOUtils.write(TEST_CHUNK_FILE_CONTENT, writer);
-    }
-    return file;
+    return writeSingleFile(Paths.get(containerData.getChunksPath()),
+        chunkFileName, TEST_CHUNK_FILE_CONTENT);
   }
 
   private File writeDbFile(
       KeyValueContainerData containerData, String dbFileName)
       throws IOException {
-    Path path = containerData.getDbFile().toPath()
-        .resolve(dbFileName);
+    return writeSingleFile(TarContainerPacker.getDbPath(containerData),
+        dbFileName, TEST_DB_FILE_CONTENT);
+  }
+
+  private File writeSingleFile(Path parentPath, String fileName,
+      String content) throws IOException {
+    Path path = parentPath.resolve(fileName);
     Files.createDirectories(path.getParent());
     File file = path.toFile();
     FileOutputStream fileStream = new FileOutputStream(file);
     try (OutputStreamWriter writer = new OutputStreamWriter(fileStream,
         UTF_8)) {
-      IOUtils.write(TEST_DB_FILE_CONTENT, writer);
+      IOUtils.write(content, writer);
     }
     return file;
   }
@@ -361,35 +363,29 @@
 
   private void assertExampleMetadataDbIsGood(Path dbPath, String filename)
       throws IOException {
-
-    Path dbFile = dbPath.resolve(filename);
-
-    Assert.assertTrue(
-        "example DB file is missing after pack/unpackContainerData: " + dbFile,
-        Files.exists(dbFile));
-
-    try (FileInputStream testFile = new FileInputStream(dbFile.toFile())) {
-      List<String> strings = IOUtils.readLines(testFile, UTF_8);
-      Assert.assertEquals(1, strings.size());
-      Assert.assertEquals(TEST_DB_FILE_CONTENT, strings.get(0));
-    }
+    assertExampleFileIsGood(dbPath, filename, TEST_DB_FILE_CONTENT);
   }
 
   private void assertExampleChunkFileIsGood(Path chunkPath, String filename)
       throws IOException {
-
-    Path chunkFile = chunkPath.resolve(filename);
-
-    Assert.assertTrue(
-        "example chunk file is missing after pack/unpackContainerData: "
-            + chunkFile,
-        Files.exists(chunkFile));
-
-    try (FileInputStream testFile = new FileInputStream(chunkFile.toFile())) {
-      List<String> strings = IOUtils.readLines(testFile, UTF_8);
-      Assert.assertEquals(1, strings.size());
-      Assert.assertEquals(TEST_CHUNK_FILE_CONTENT, strings.get(0));
-    }
+    assertExampleFileIsGood(chunkPath, filename, TEST_CHUNK_FILE_CONTENT);
   }
 
+  private void assertExampleFileIsGood(Path parentPath, String filename,
+      String content) throws IOException {
+
+    Path exampleFile = parentPath.resolve(filename);
+
+    Assert.assertTrue(
+        "example file is missing after pack/unpackContainerData: "
+            + exampleFile,
+        Files.exists(exampleFile));
+
+    try (FileInputStream testFile =
+             new FileInputStream(exampleFile.toFile())) {
+      List<String> strings = IOUtils.readLines(testFile, UTF_8);
+      Assert.assertEquals(1, strings.size());
+      Assert.assertEquals(content, strings.get(0));
+    }
+  }
 }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java
index 5773eb3..377c780 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java
@@ -26,13 +26,16 @@
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
-import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo;
+import org.apache.hadoop.ozone.container.keyvalue.ContainerTestVersionInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -73,23 +76,28 @@
   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);
 
     volumeSet = mock(MutableVolumeSet.class);
 
@@ -137,6 +145,11 @@
 
   }
 
+  @After
+  public void cleanup() {
+    BlockUtils.shutdownCache(config);
+  }
+
   @Test
   public void testPutBlock() throws Exception {
     assertEquals(0, keyValueContainer.getContainerData().getBlockCount());
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestContainerReader.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestContainerReader.java
index 15fe50a..2ab6f95 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestContainerReader.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestContainerReader.java
@@ -31,20 +31,25 @@
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
+import org.apache.hadoop.ozone.container.keyvalue.ContainerTestVersionInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.mockito.Mockito;
 
 import java.io.File;
@@ -52,6 +57,7 @@
 import java.util.List;
 import java.util.UUID;
 
+import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.createDbInstancesForTestIfNeeded;
 import static org.mockito.ArgumentMatchers.anyList;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.mock;
@@ -59,6 +65,7 @@
 /**
  * Test ContainerReader class which loads containers from disks.
  */
+@RunWith(Parameterized.class)
 public class TestContainerReader {
 
   @Rule
@@ -76,18 +83,35 @@
   private int blockCount = 10;
   private long blockLen = 1024;
 
+  private final ContainerLayoutVersion layout;
+  private String schemaVersion;
+
+  public TestContainerReader(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    this.conf = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, conf);
+  }
+
+
+  @Parameterized.Parameters
+  public static Iterable<Object[]> parameters() {
+    return ContainerTestVersionInfo.versionParameters();
+  }
+
   @Before
   public void setup() throws Exception {
 
     File volumeDir = tempDir.newFolder();
     volumeSet = Mockito.mock(MutableVolumeSet.class);
     containerSet = new ContainerSet();
-    conf = new OzoneConfiguration();
 
     datanodeId = UUID.randomUUID();
     hddsVolume = new HddsVolume.Builder(volumeDir
         .getAbsolutePath()).conf(conf).datanodeUuid(datanodeId
         .toString()).clusterID(clusterId).build();
+    StorageVolumeUtil.checkVolume(hddsVolume, clusterId, clusterId, conf,
+        null, null);
 
     volumeSet = mock(MutableVolumeSet.class);
     volumeChoosingPolicy = mock(RoundRobinVolumeChoosingPolicy.class);
@@ -96,7 +120,7 @@
 
     for (int i = 0; i < 2; i++) {
       KeyValueContainerData keyValueContainerData = new KeyValueContainerData(i,
-          ContainerLayoutVersion.FILE_PER_BLOCK,
+          layout,
           (long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(),
           datanodeId.toString());
 
@@ -114,28 +138,33 @@
         blkNames = addBlocks(keyValueContainer, false);
         markBlocksForDelete(keyValueContainer, false, blkNames, i);
       }
-      // Close the RocksDB instance for this container and remove from the cache
-      // so it does not affect the ContainerReader, which avoids using the cache
-      // at startup for performance reasons.
-      BlockUtils.removeDB(keyValueContainerData, conf);
     }
+    // Close the RocksDB instance for this container and remove from the cache
+    // so it does not affect the ContainerReader, which avoids using the cache
+    // at startup for performance reasons.
+    ContainerCache.getInstance(conf).shutdownCache();
   }
 
+  @After
+  public void cleanup() {
+    BlockUtils.shutdownCache(conf);
+  }
 
   private void markBlocksForDelete(KeyValueContainer keyValueContainer,
       boolean setMetaData, List<Long> blockNames, int count) throws Exception {
-    try (ReferenceCountedDB metadataStore = BlockUtils.getDB(keyValueContainer
-        .getContainerData(), conf)) {
+    KeyValueContainerData cData = keyValueContainer.getContainerData();
+    try (DBHandle metadataStore = BlockUtils.getDB(cData, conf)) {
 
       for (int i = 0; i < count; i++) {
         Table<String, BlockData> blockDataTable =
                 metadataStore.getStore().getBlockDataTable();
 
-        String blk = Long.toString(blockNames.get(i));
+        Long localID = blockNames.get(i);
+        String blk = cData.blockKey(localID);
         BlockData blkInfo = blockDataTable.get(blk);
 
         blockDataTable.delete(blk);
-        blockDataTable.put(OzoneConsts.DELETING_KEY_PREFIX + blk, blkInfo);
+        blockDataTable.put(cData.deletingBlockKey(localID), blkInfo);
       }
 
       if (setMetaData) {
@@ -143,7 +172,7 @@
         // and bytes used metadata values, so those do not change.
         Table<String, Long> metadataTable =
                 metadataStore.getStore().getMetadataTable();
-        metadataTable.put(OzoneConsts.PENDING_DELETE_BLOCK_COUNT, (long)count);
+        metadataTable.put(cData.pendingDeleteBlockCountKey(), (long)count);
       }
     }
 
@@ -152,10 +181,9 @@
   private List<Long> addBlocks(KeyValueContainer keyValueContainer,
       boolean setMetaData) throws Exception {
     long containerId = keyValueContainer.getContainerData().getContainerID();
-
+    KeyValueContainerData cData = keyValueContainer.getContainerData();
     List<Long> blkNames = new ArrayList<>();
-    try (ReferenceCountedDB metadataStore = BlockUtils.getDB(keyValueContainer
-        .getContainerData(), conf)) {
+    try (DBHandle metadataStore = BlockUtils.getDB(cData, conf)) {
 
       for (int i = 0; i < blockCount; i++) {
         // Creating BlockData
@@ -172,14 +200,14 @@
         blockData.setChunks(chunkList);
         blkNames.add(localBlockID);
         metadataStore.getStore().getBlockDataTable()
-                .put(Long.toString(localBlockID), blockData);
+                .put(cData.blockKey(localBlockID), blockData);
       }
 
       if (setMetaData) {
         metadataStore.getStore().getMetadataTable()
-                .put(OzoneConsts.BLOCK_COUNT, (long)blockCount);
+                .put(cData.blockCountKey(), (long)blockCount);
         metadataStore.getStore().getMetadataTable()
-                .put(OzoneConsts.CONTAINER_BYTES_USED, blockCount * blockLen);
+                .put(cData.bytesUsedKey(), blockCount * blockLen);
       }
     }
 
@@ -228,6 +256,8 @@
     hddsVolume1 = new HddsVolume.Builder(volumeDir1
         .getAbsolutePath()).conf(conf).datanodeUuid(datanode
         .toString()).clusterID(clusterId).build();
+    StorageVolumeUtil.checkVolume(hddsVolume1, clusterId, clusterId, conf,
+        null, null);
     volumeChoosingPolicy1 = mock(RoundRobinVolumeChoosingPolicy.class);
     Mockito.when(volumeChoosingPolicy1.chooseVolume(anyList(), anyLong()))
         .thenReturn(hddsVolume1);
@@ -235,13 +265,12 @@
     int containerCount = 3;
     for (int i = 0; i < containerCount; i++) {
       KeyValueContainerData keyValueContainerData = new KeyValueContainerData(i,
-          ContainerLayoutVersion.FILE_PER_BLOCK,
+          layout,
           (long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(),
           datanodeId.toString());
       KeyValueContainer keyValueContainer =
           new KeyValueContainer(keyValueContainerData, conf);
       keyValueContainer.create(volumeSet1, volumeChoosingPolicy1, clusterId);
-      BlockUtils.removeDB(keyValueContainerData, conf);
 
       if (i == 0) {
         // rename first container directory name
@@ -252,6 +281,7 @@
         Assert.assertTrue(containerPath.renameTo(new File(renamePath)));
       }
     }
+    ContainerCache.getInstance(conf).shutdownCache();
 
     ContainerReader containerReader = new ContainerReader(volumeSet1,
         hddsVolume1, containerSet1, conf);
@@ -268,6 +298,8 @@
       volumeDirs[i] = tempDir.newFolder();
       datanodeDirs = datanodeDirs.append(volumeDirs[i]).append(",");
     }
+
+    BlockUtils.shutdownCache(conf);
     conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY,
         datanodeDirs.toString());
     conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR,
@@ -275,8 +307,9 @@
     MutableVolumeSet volumeSets =
         new MutableVolumeSet(datanodeId.toString(), clusterId, conf, null,
             StorageVolume.VolumeType.DATA_VOLUME, null);
+    createDbInstancesForTestIfNeeded(volumeSets, clusterId, clusterId, conf);
     ContainerCache cache = ContainerCache.getInstance(conf);
-    cache.clear();
+    cache.shutdownCache();
 
     RoundRobinVolumeChoosingPolicy policy =
         new RoundRobinVolumeChoosingPolicy();
@@ -285,7 +318,7 @@
     blockCount = containerCount;
     for (int i = 0; i < containerCount; i++) {
       KeyValueContainerData keyValueContainerData =
-          new KeyValueContainerData(i, ContainerLayoutVersion.FILE_PER_BLOCK,
+          new KeyValueContainerData(i, layout,
               (long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(),
               datanodeId.toString());
 
@@ -302,11 +335,11 @@
         blkNames = addBlocks(keyValueContainer, false);
         markBlocksForDelete(keyValueContainer, false, blkNames, i);
       }
-      // Close the RocksDB instance for this container and remove from the cache
-      // so it does not affect the ContainerReader, which avoids using the cache
-      // at startup for performance reasons.
-      BlockUtils.removeDB(keyValueContainerData, conf);
     }
+    // Close the RocksDB instance for this container and remove from the cache
+    // so it does not affect the ContainerReader, which avoids using the cache
+    // at startup for performance reasons.
+    cache.shutdownCache();
 
     List<StorageVolume> volumes = volumeSets.getVolumesList();
     ContainerReader[] containerReaders = new ContainerReader[volumeNum];
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index fec52c8..5ae3bad 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -30,20 +30,20 @@
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.utils.db.Table;
-import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
-import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo;
+import org.apache.hadoop.ozone.container.keyvalue.ContainerTestVersionInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
@@ -60,6 +60,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.util.Random;
 import java.util.UUID;
 import java.util.HashMap;
@@ -67,6 +68,7 @@
 import java.util.ArrayList;
 
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.DISK_OUT_OF_SPACE;
+import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.createDbInstancesForTestIfNeeded;
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -92,31 +94,37 @@
   private final int numTestContainers = 10;
 
   private final ContainerLayoutVersion layout;
+  private final String schemaVersion;
 
-  public TestOzoneContainer(ContainerLayoutVersion layout) {
-    this.layout = layout;
+  public TestOzoneContainer(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    this.conf = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, conf);
   }
 
   @Parameterized.Parameters
   public static Iterable<Object[]> parameters() {
-    return ContainerLayoutTestInfo.containerLayoutParameters();
+    return ContainerTestVersionInfo.versionParameters();
   }
 
   @Before
   public void setUp() throws Exception {
-    conf = new OzoneConfiguration();
     conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, folder.getRoot()
         .getAbsolutePath());
     conf.set(HddsConfigKeys.OZONE_METADATA_DIRS,
         folder.newFolder().getAbsolutePath());
     commitSpaceMap = new HashMap<String, Long>();
-    volumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf,
-        null, StorageVolume.VolumeType.DATA_VOLUME, null);
+    volumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(),
+        clusterId, conf, null, StorageVolume.VolumeType.DATA_VOLUME, null);
+    createDbInstancesForTestIfNeeded(volumeSet, clusterId, clusterId, conf);
     volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
   }
 
   @After
   public void cleanUp() throws Exception {
+    BlockUtils.shutdownCache(conf);
+
     if (volumeSet != null) {
       volumeSet.shutdown();
       volumeSet = null;
@@ -158,8 +166,8 @@
       Preconditions.checkState(freeBytes >= 0);
       commitSpaceMap.put(getVolumeKey(myVolume),
           Long.valueOf(volCommitBytes + freeBytes));
-      BlockUtils.removeDB(keyValueContainerData, conf);
     }
+    BlockUtils.shutdownCache(conf);
 
     DatanodeStateMachine stateMachine = Mockito.mock(
         DatanodeStateMachine.class);
@@ -185,6 +193,17 @@
     conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR,
             String.join(",",
             path + "/ratis1", path + "/ratis2", path + "ratis3"));
+
+    File[] dbPaths = new File[3];
+    StringBuilder dbDirString = new StringBuilder();
+    for (int i = 0; i < 3; i++) {
+      dbPaths[i] = folder.newFolder();
+      dbDirString.append(dbPaths[i]).append(",");
+    }
+    conf.set(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR,
+        dbDirString.toString());
+    ContainerTestUtils.enableSchemaV3(conf);
+
     DatanodeStateMachine stateMachine = Mockito.mock(
             DatanodeStateMachine.class);
     StateContext context = Mockito.mock(StateContext.class);
@@ -200,7 +219,8 @@
     Assert.assertEquals(3,
             ozoneContainer.getNodeReport().getMetadataStorageReportList()
                     .size());
-
+    Assert.assertEquals(3,
+            ozoneContainer.getNodeReport().getDbStorageReportList().size());
   }
 
   @Test
@@ -231,7 +251,7 @@
         StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList());
     // Format the volumes
     for (HddsVolume volume : volumes) {
-      volume.format(UUID.randomUUID().toString());
+      volume.format(clusterId);
 
       // eat up all available space except size of 1 container
       volume.incCommittedBytes(volume.getAvailable() - containerSize);
@@ -277,36 +297,37 @@
 
     long freeBytes = container.getContainerData().getMaxSize();
     long containerId = container.getContainerData().getContainerID();
-    ReferenceCountedDB db = BlockUtils.getDB(container
-        .getContainerData(), conf);
+    KeyValueContainerData cData = container.getContainerData();
+    try (DBHandle db = BlockUtils.getDB(cData, conf)) {
 
-    Table<String, Long> metadataTable = db.getStore().getMetadataTable();
-    Table<String, BlockData> blockDataTable = db.getStore().getBlockDataTable();
+      Table<String, Long> metadataTable =
+          db.getStore().getMetadataTable();
+      Table<String, BlockData> blockDataTable =
+          db.getStore().getBlockDataTable();
 
-    for (int bi = 0; bi < blocks; bi++) {
-      // Creating BlockData
-      BlockID blockID = new BlockID(containerId, bi);
-      BlockData blockData = new BlockData(blockID);
-      List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
+      for (int bi = 0; bi < blocks; bi++) {
+        // Creating BlockData
+        BlockID blockID = new BlockID(containerId, bi);
+        BlockData blockData = new BlockData(blockID);
+        List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
 
-      chunkList.clear();
-      for (int ci = 0; ci < chunksPerBlock; ci++) {
-        String chunkName = strBlock + bi + strChunk + ci;
-        long offset = ci * (long) datalen;
-        ChunkInfo info = new ChunkInfo(chunkName, offset, datalen);
-        usedBytes += datalen;
-        chunkList.add(info.getProtoBufMessage());
+        chunkList.clear();
+        for (int ci = 0; ci < chunksPerBlock; ci++) {
+          String chunkName = strBlock + bi + strChunk + ci;
+          long offset = ci * (long) datalen;
+          ChunkInfo info = new ChunkInfo(chunkName, offset, datalen);
+          usedBytes += datalen;
+          chunkList.add(info.getProtoBufMessage());
+        }
+        blockData.setChunks(chunkList);
+        blockDataTable.put(cData.blockKey(blockID.getLocalID()), blockData);
       }
-      blockData.setChunks(chunkList);
-      blockDataTable.put(Long.toString(blockID.getLocalID()), blockData);
+
+      // Set Block count and used bytes.
+      metadataTable.put(cData.blockCountKey(), (long) blocks);
+      metadataTable.put(cData.bytesUsedKey(), usedBytes);
     }
-
-    // Set Block count and used bytes.
-    metadataTable.put(OzoneConsts.BLOCK_COUNT, (long)blocks);
-    metadataTable.put(OzoneConsts.CONTAINER_BYTES_USED, usedBytes);
-
     // remaining available capacity of the container
-    db.close();
     return (freeBytes - usedBytes);
   }
 
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToSchemaV3.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToSchemaV3.java
new file mode 100644
index 0000000..a7e61c5
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToSchemaV3.java
@@ -0,0 +1,759 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.upgrade;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipeline;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
+import org.apache.hadoop.ozone.container.common.DatanodeLayoutStorage;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.hadoop.ozone.container.common.ScmTestMock;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
+import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
+import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask;
+import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.DbVolume;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import static org.mockito.ArgumentMatchers.anyObject;
+
+/**
+ * Tests upgrading a single datanode from container Schema V2 to Schema V3.
+ */
+@RunWith(Parameterized.class)
+public class TestDatanodeUpgradeToSchemaV3 {
+  @Rule
+  public TemporaryFolder tempFolder;
+
+  private DatanodeStateMachine dsm;
+  private final OzoneConfiguration conf;
+  private static final String CLUSTER_ID = "clusterID";
+  private final boolean schemaV3Enabled;
+
+  private RPC.Server scmRpcServer;
+  private InetSocketAddress address;
+  private ScmTestMock scmServerImpl;
+
+  private Random random;
+
+  // hdds.datanode.container.schema.v3.enabled
+  @Parameterized.Parameters
+  public static Collection<Object[]> getSchemaFiles() {
+    Collection<Object[]> parameters = new ArrayList<>();
+    parameters.add(new Boolean[]{false});
+    parameters.add(new Boolean[]{true});
+    return parameters;
+  }
+
+  public TestDatanodeUpgradeToSchemaV3(Boolean enable) {
+    this.schemaV3Enabled = enable;
+    conf = new OzoneConfiguration();
+    conf.setBoolean(DatanodeConfiguration.CONTAINER_SCHEMA_V3_ENABLED,
+        this.schemaV3Enabled);
+  }
+
+  @Before
+  public void setup() throws Exception {
+    tempFolder = new TemporaryFolder();
+    tempFolder.create();
+    random = new Random();
+
+    address = SCMTestUtils.getReuseableAddress();
+    conf.setSocketAddr(ScmConfigKeys.OZONE_SCM_NAMES, address);
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS,
+        tempFolder.getRoot().getAbsolutePath());
+  }
+
+  @After
+  public void teardown() throws Exception {
+    if (scmRpcServer != null) {
+      scmRpcServer.stop();
+    }
+
+    if (dsm != null) {
+      dsm.close();
+    }
+  }
+
+  /**
+   * Test RocksDB is created on data volume, not matter Schema V3 is
+   * enabled or not.
+   * If Schema V3 is enabled, RocksDB will be loaded.
+   */
+  @Test
+  public void testDBOnHddsVolume() throws Exception {
+    // start DN and SCM
+    startScmServer();
+    addHddsVolume();
+
+    startPreFinalizedDatanode();
+    HddsVolume dataVolume = (HddsVolume) dsm.getContainer().getVolumeSet()
+        .getVolumesList().get(0);
+    Assert.assertNull(dataVolume.getDbVolume());
+    Assert.assertFalse(dataVolume.isDbLoaded());
+
+    dsm.finalizeUpgrade();
+    // RocksDB is created during upgrade
+    File dbFile = new File(dataVolume.getStorageDir().getAbsolutePath() + "/" +
+        dataVolume.getClusterID() + "/" + dataVolume.getStorageID());
+    Assert.assertTrue(dbFile.exists());
+
+    // RocksDB loaded when SchemaV3 is enabled
+    if (VersionedDatanodeFeatures.SchemaV3.isFinalizedAndEnabled(conf)) {
+      Assert.assertTrue(dataVolume.getDbParentDir().getAbsolutePath()
+          .startsWith(dataVolume.getStorageDir().toString()));
+    } else {
+      // RocksDB is not loaded when SchemaV3 is disabled.
+      Assert.assertFalse(dataVolume.isDbLoaded());
+    }
+  }
+
+  /**
+   * Test RocksDB is created on DB volume when configured, not matter
+   * Schema V3 is enabled or not.
+   * If Schema V3 is enabled, RocksDB will be loaded.
+   */
+  @Test
+  public void testDBOnDbVolume() throws Exception {
+    // start DN and SCM
+    startScmServer();
+    addHddsVolume();
+    addDbVolume();
+
+    startPreFinalizedDatanode();
+    HddsVolume dataVolume = (HddsVolume) dsm.getContainer().getVolumeSet()
+        .getVolumesList().get(0);
+    Assert.assertNull(dataVolume.getDbParentDir());
+
+    dsm.finalizeUpgrade();
+    // RocksDB is created during upgrade
+    DbVolume dbVolume = (DbVolume) dsm.getContainer().getDbVolumeSet()
+        .getVolumesList().get(0);
+    Assert.assertEquals(dbVolume, dataVolume.getDbVolume());
+    Assert.assertTrue(
+        dbVolume.getHddsVolumeIDs().contains(dataVolume.getStorageID()));
+    File dbFile = new File(dbVolume.getStorageDir().getAbsolutePath() + "/" +
+        dbVolume.getClusterID() + "/" + dataVolume.getStorageID());
+    Assert.assertTrue(dbFile.exists());
+
+    // RocksDB loaded when SchemaV3 is enabled
+    if (VersionedDatanodeFeatures.SchemaV3.isFinalizedAndEnabled(conf)) {
+      Assert.assertTrue(dataVolume.getDbParentDir().getAbsolutePath()
+          .startsWith(dbVolume.getStorageDir().toString()));
+    } else {
+      // RocksDB is not loaded when SchemaV3 is disabled.
+      Assert.assertFalse(dataVolume.isDbLoaded());
+    }
+  }
+
+  /**
+   * Test RocksDB in created in Finalize action for an existing hddsVolume.
+   * This mimics the real cluster upgrade situation.
+   */
+  @Test
+  public void testDBCreatedInFinalize() throws Exception {
+    // start DN and SCM
+    startScmServer();
+    // add one HddsVolume
+    addHddsVolume();
+
+    // Set layout version.
+    DatanodeLayoutStorage layoutStorage = new DatanodeLayoutStorage(conf,
+        UUID.randomUUID().toString(),
+        HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion());
+    layoutStorage.initialize();
+    dsm = new DatanodeStateMachine(
+        ContainerTestUtils.createDatanodeDetails(), conf, null, null, null);
+    HddsVolume dataVolume = (
+        HddsVolume) dsm.getContainer().getVolumeSet().getVolumesList().get(0);
+    // Format HddsVolume to mimic the real cluster upgrade situation
+    dataVolume.format(CLUSTER_ID);
+    File idDir = new File(dataVolume.getStorageDir(), CLUSTER_ID);
+    if (!idDir.mkdir()) {
+      Assert.fail("Failed to create id directory");
+    }
+
+    Assert.assertNull(dataVolume.getDbParentDir());
+
+    // Restart DN and finalize upgrade
+    restartDatanode(
+        HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion(), true);
+    dsm.finalizeUpgrade();
+
+    // RocksDB is created by upgrade action
+    dataVolume = ((HddsVolume) dsm.getContainer().getVolumeSet()
+        .getVolumesList().get(0));
+    Assert.assertNotNull(dataVolume.getDbParentDir());
+    if (VersionedDatanodeFeatures.SchemaV3.isFinalizedAndEnabled(conf)) {
+      Assert.assertTrue(dataVolume.isDbLoaded());
+    } else {
+      Assert.assertFalse(dataVolume.isDbLoaded());
+    }
+  }
+
+  /**
+   * Test finalize twice won't recreate any RocksDB for HddsVolume.
+   */
+  @Test
+  public void testFinalizeTwice() throws Exception {
+    // start DN and SCM
+    startScmServer();
+    // add one HddsVolume and two DbVolume
+    addHddsVolume();
+    addDbVolume();
+    addDbVolume();
+
+    startPreFinalizedDatanode();
+    dsm.finalizeUpgrade();
+
+    DbVolume dbVolume = ((HddsVolume) dsm.getContainer().getVolumeSet()
+        .getVolumesList().get(0)).getDbVolume();
+    Assert.assertNotNull(dbVolume);
+
+    dsm.finalizeUpgrade();
+    // DB Dir should be the same.
+    Assert.assertEquals(dbVolume, ((HddsVolume) dsm.getContainer()
+        .getVolumeSet().getVolumesList().get(0)).getDbVolume());
+  }
+
+  /**
+   * For a finalized cluster, add a new HddsVolume.
+   */
+  @Test
+  public void testAddHddsVolumeAfterFinalize() throws Exception {
+    // start DN and SCM
+    startScmServer();
+    addHddsVolume();
+
+    startPreFinalizedDatanode();
+    dsm.finalizeUpgrade();
+
+    // Add a new HddsVolume. It should have DB created after DN restart.
+    addHddsVolume();
+    restartDatanode(HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), true);
+    for (StorageVolume vol:
+        dsm.getContainer().getVolumeSet().getVolumesList()) {
+      HddsVolume hddsVolume = (HddsVolume) vol;
+      if (VersionedDatanodeFeatures.SchemaV3.isFinalizedAndEnabled(conf)) {
+        Assert.assertTrue(hddsVolume.isDbLoaded());
+        Assert.assertTrue(hddsVolume.getDbParentDir().getAbsolutePath()
+            .startsWith(hddsVolume.getStorageDir().getAbsolutePath()));
+      } else {
+        Assert.assertFalse(hddsVolume.isDbLoaded());
+      }
+    }
+  }
+
+  /**
+   * For a finalized cluster, add a new DbVolume.
+   */
+  @Test
+  public void testAddDbVolumeAfterFinalize() throws Exception {
+    startScmServer();
+    addHddsVolume();
+
+    startPreFinalizedDatanode();
+    HddsVolume hddsVolume = (HddsVolume) dsm.getContainer().getVolumeSet()
+        .getVolumesList().get(0);
+    Assert.assertNull(hddsVolume.getDbParentDir());
+    dsm.finalizeUpgrade();
+    // DB is created during upgrade
+    File dbDir = hddsVolume.getDbParentDir();
+    Assert.assertTrue(dbDir.getAbsolutePath().startsWith(
+        hddsVolume.getStorageDir().getAbsolutePath()));
+
+    // Add a new DbVolume
+    addDbVolume();
+    restartDatanode(HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), true);
+
+    // HddsVolume should still use the rocksDB under it's volume
+    DbVolume dbVolume = (DbVolume) dsm.getContainer().getDbVolumeSet()
+        .getVolumesList().get(0);
+    Assert.assertEquals(0, dbVolume.getHddsVolumeIDs().size());
+
+    if (VersionedDatanodeFeatures.SchemaV3.isFinalizedAndEnabled(conf)) {
+      hddsVolume = (HddsVolume) dsm.getContainer().getVolumeSet()
+          .getVolumesList().get(0);
+      Assert.assertEquals(dbDir, hddsVolume.getDbParentDir());
+      Assert.assertTrue(hddsVolume.isDbLoaded());
+    }
+  }
+
+  /**
+   * For a finalized cluster, add a new DbVolume and a new HddsVolume.
+   */
+  @Test
+  public void testAddDbAndHddsVolumeAfterFinalize() throws Exception {
+    // start DN and SCM
+    startScmServer();
+    addHddsVolume();
+
+    startPreFinalizedDatanode();
+    dsm.finalizeUpgrade();
+
+    addDbVolume();
+    File newDataVolume = addHddsVolume();
+    restartDatanode(HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), true);
+
+    DbVolume dbVolume = (DbVolume) dsm.getContainer().getDbVolumeSet()
+        .getVolumesList().get(0);
+
+    for (StorageVolume vol:
+        dsm.getContainer().getVolumeSet().getVolumesList()) {
+      HddsVolume hddsVolume = (HddsVolume) vol;
+      File dbFile;
+      if (hddsVolume.getStorageDir().getAbsolutePath().startsWith(
+          newDataVolume.getAbsolutePath())) {
+        if (VersionedDatanodeFeatures.SchemaV3.isFinalizedAndEnabled(conf)) {
+          Assert.assertEquals(dbVolume, hddsVolume.getDbVolume());
+        }
+        // RocksDB of newly added HddsVolume is created on the newly added
+        // DbVolume
+        dbFile = new File(dbVolume.getStorageDir() + "/" +
+            hddsVolume.getClusterID() + "/" + hddsVolume.getStorageID());
+      } else {
+        Assert.assertNull(hddsVolume.getDbVolume());
+        dbFile = new File(hddsVolume.getStorageDir() + "/" +
+            hddsVolume.getClusterID() + "/" + hddsVolume.getStorageID());
+      }
+      if (VersionedDatanodeFeatures.SchemaV3.isFinalizedAndEnabled(conf)) {
+        Assert.assertTrue(hddsVolume.isDbLoaded());
+        Assert.assertTrue(hddsVolume.getDbParentDir().exists());
+        Assert.assertTrue(dbFile.exists());
+        Assert.assertEquals(dbFile, hddsVolume.getDbParentDir());
+      }
+    }
+  }
+
+  /**
+   * Test data write after finalization.
+   */
+  @Test
+  public void testWriteWithV3Enabled() throws Exception {
+    testWrite(false, OzoneConsts.SCHEMA_V2);
+  }
+
+  /**
+   * Test data write after finalization.
+   */
+  @Test
+  public void testWriteWithV3Disabled() throws Exception {
+    testWrite(true, OzoneConsts.SCHEMA_V3);
+  }
+
+  public void testWrite(boolean enable, String expectedVersion)
+      throws Exception {
+    // start DN and SCM
+    startScmServer();
+    addHddsVolume();
+    // Disable Schema V3
+    conf.setBoolean(DatanodeConfiguration.CONTAINER_SCHEMA_V3_ENABLED, false);
+    startPreFinalizedDatanode();
+    dsm.finalizeUpgrade();
+
+    final Pipeline pipeline = getPipeline();
+    // Create a container to write data.
+    final long containerID1 = addContainer(pipeline);
+    putBlock(containerID1, pipeline);
+    closeContainer(containerID1, pipeline);
+    KeyValueContainer container = (KeyValueContainer)
+        dsm.getContainer().getContainerSet().getContainer(containerID1);
+    // When SchemaV3 is disabled, new data should be saved as SchemaV2.
+    Assert.assertEquals(OzoneConsts.SCHEMA_V2,
+        container.getContainerData().getSchemaVersion());
+
+    // Set SchemaV3 enable status
+    conf.setBoolean(DatanodeConfiguration.CONTAINER_SCHEMA_V3_ENABLED,
+        enable);
+    restartDatanode(HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), true);
+
+    // Write new data
+    final long containerID2 = addContainer(pipeline);
+    putBlock(containerID2, pipeline);
+    closeContainer(containerID2, pipeline);
+    container = (KeyValueContainer)
+        dsm.getContainer().getContainerSet().getContainer(containerID2);
+    // If SchemaV3 is enabled, new data should be saved as SchemaV3
+    // If SchemaV3 is still disabled, new data should still be saved as SchemaV2
+    Assert.assertEquals(expectedVersion,
+        container.getContainerData().getSchemaVersion());
+  }
+
+  /**
+   * Test data read during and after finalization.
+   */
+  @Test
+  public void testReadsDuringFinalize() throws Exception {
+    // start DN and SCM
+    startScmServer();
+    addHddsVolume();
+    startPreFinalizedDatanode();
+    final Pipeline pipeline = getPipeline();
+
+    // Add data to read.
+    final long containerID = addContainer(pipeline);
+    ContainerProtos.WriteChunkRequestProto writeChunk = putBlock(containerID,
+        pipeline);
+    closeContainer(containerID, pipeline);
+
+    // Create thread to keep reading during finalization.
+    ExecutorService executor = Executors.newFixedThreadPool(1);
+    Future<Void> readFuture = executor.submit(() -> {
+      // Layout version check should be thread safe.
+      while (!dsm.getLayoutVersionManager()
+          .isAllowed(HDDSLayoutFeature.DATANODE_SCHEMA_V3)) {
+        readChunk(writeChunk, pipeline);
+      }
+      // Make sure we can read after finalizing too.
+      readChunk(writeChunk, pipeline);
+      return null;
+    });
+
+    dsm.finalizeUpgrade();
+    // If there was a failure reading during the upgrade, the exception will
+    // be thrown here.
+    readFuture.get();
+  }
+
+  /**
+   * Test finalization failure.
+   */
+  @Test
+  public void testFinalizeFailure() throws Exception {
+    // start DN and SCM
+    startScmServer();
+    addHddsVolume();
+    // Let HddsVolume be formatted to mimic the real cluster upgrade
+    // Set layout version.
+    DatanodeLayoutStorage layoutStorage = new DatanodeLayoutStorage(conf,
+        UUID.randomUUID().toString(),
+        HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion());
+    layoutStorage.initialize();
+    dsm = new DatanodeStateMachine(
+        ContainerTestUtils.createDatanodeDetails(), conf, null, null, null);
+    HddsVolume dataVolume = (
+        HddsVolume) dsm.getContainer().getVolumeSet().getVolumesList().get(0);
+    // Format HddsVolume to mimic the real cluster upgrade situation
+    dataVolume.format(CLUSTER_ID);
+    File idDir = new File(dataVolume.getStorageDir(), CLUSTER_ID);
+    if (!idDir.mkdir()) {
+      Assert.fail("Failed to create id directory");
+    }
+    Assert.assertNull(dataVolume.getDbParentDir());
+
+    // Restart DN
+    restartDatanode(
+        HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion(), true);
+
+    // Write some data.
+    final Pipeline pipeline = getPipeline();
+    final long containerID = addContainer(pipeline);
+    ContainerProtos.WriteChunkRequestProto writeChunk = putBlock(containerID,
+        pipeline);
+    closeContainer(containerID, pipeline);
+    KeyValueContainer container = (KeyValueContainer)
+        dsm.getContainer().getContainerSet().getContainer(containerID);
+    Assert.assertEquals(OzoneConsts.SCHEMA_V2,
+        container.getContainerData().getSchemaVersion());
+
+
+    HddsVolume volume = Mockito.mock(HddsVolume.class);
+    Mockito.doThrow(new IOException("Failed to init DB")).when(volume).
+        createDbStore(anyObject());
+    Map volumeMap = new HashMap<String, StorageVolume>();
+    volumeMap.put(dataVolume.getStorageID(), volume);
+    dsm.getContainer().getVolumeSet().setVolumeMap(volumeMap);
+
+    // Finalize will fail because of DB creation failure
+    try {
+      dsm.finalizeUpgrade();
+    } catch (Exception e) {
+      // Currently there will be retry if finalization failed.
+      // Let's assume retry is terminated by user.
+    }
+
+    // Check that old data is readable
+    container = (KeyValueContainer)
+        dsm.getContainer().getContainerSet().getContainer(containerID);
+    Assert.assertEquals(OzoneConsts.SCHEMA_V2,
+        container.getContainerData().getSchemaVersion());
+    readChunk(writeChunk, pipeline);
+
+    // SchemaV3 is not finalized, so still ERASURE_CODED_STORAGE_SUPPORT
+    restartDatanode(
+        HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion(), true);
+
+    // Old data is readable after DN restart
+    container = (KeyValueContainer)
+        dsm.getContainer().getContainerSet().getContainer(containerID);
+    Assert.assertEquals(OzoneConsts.SCHEMA_V2,
+        container.getContainerData().getSchemaVersion());
+    readChunk(writeChunk, pipeline);
+  }
+
+  public void checkContainerPathID(long containerID, String expectedID) {
+    KeyValueContainerData data =
+        (KeyValueContainerData) dsm.getContainer().getContainerSet()
+            .getContainer(containerID).getContainerData();
+    Assert.assertTrue(data.getChunksPath().contains(expectedID));
+    Assert.assertTrue(data.getMetadataPath().contains(expectedID));
+  }
+
+  public List<File> getHddsSubdirs(File volume) {
+    File[] subdirsArray = getHddsRoot(volume).listFiles(File::isDirectory);
+    Assert.assertNotNull(subdirsArray);
+    return Arrays.asList(subdirsArray);
+  }
+
+  public File getHddsRoot(File volume) {
+    return new File(HddsVolumeUtil.getHddsRoot(volume.getAbsolutePath()));
+  }
+
+  /**
+   * Starts the datanode with the fore layout version, and calls the version
+   * endpoint task to get cluster ID and SCM ID.
+   *
+   * The daemon for the datanode state machine is not started in this test.
+   * This greatly speeds up execution time.
+   * It means we do not have heartbeat functionality or pre-finalize
+   * upgrade actions, but neither of those things are needed for these tests.
+   */
+  public void startPreFinalizedDatanode() throws Exception {
+    // Set layout version.
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS,
+        tempFolder.getRoot().getAbsolutePath());
+    DatanodeLayoutStorage layoutStorage = new DatanodeLayoutStorage(conf,
+        UUID.randomUUID().toString(),
+        HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion());
+    layoutStorage.initialize();
+
+    // Build and start the datanode.
+    DatanodeDetails dd = ContainerTestUtils.createDatanodeDetails();
+    DatanodeStateMachine newDsm = new DatanodeStateMachine(dd,
+        conf, null, null, null);
+    int actualMlv = newDsm.getLayoutVersionManager().getMetadataLayoutVersion();
+    Assert.assertEquals(
+        HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion(),
+        actualMlv);
+    if (dsm != null) {
+      dsm.close();
+    }
+    dsm = newDsm;
+
+    callVersionEndpointTask();
+  }
+
+  public void restartDatanode(int expectedMlv, boolean exactMatch)
+      throws Exception {
+    // Stop existing datanode.
+    DatanodeDetails dd = dsm.getDatanodeDetails();
+    dsm.close();
+
+    // Start new datanode with the same configuration.
+    dsm = new DatanodeStateMachine(dd,
+        conf, null, null, null);
+    int mlv = dsm.getLayoutVersionManager().getMetadataLayoutVersion();
+    if (exactMatch) {
+      Assert.assertEquals(expectedMlv, mlv);
+    } else {
+      Assert.assertTrue("Expected minimum mlv(" + expectedMlv
+          + ") is smaller than mlv(" + mlv + ").", expectedMlv <= mlv);
+    }
+
+    callVersionEndpointTask();
+  }
+
+  /**
+   * Get the cluster ID and SCM ID from SCM to the datanode.
+   */
+  public void callVersionEndpointTask() throws Exception {
+    try (EndpointStateMachine esm = ContainerTestUtils.createEndpoint(conf,
+        address, 1000)) {
+      VersionEndpointTask vet = new VersionEndpointTask(esm, conf,
+          dsm.getContainer());
+      esm.setState(EndpointStateMachine.EndPointStates.GETVERSION);
+      vet.call();
+    }
+  }
+
+  public String startScmServer() throws IOException {
+    String scmID = UUID.randomUUID().toString();
+    scmServerImpl = new ScmTestMock(CLUSTER_ID, scmID);
+    scmRpcServer = SCMTestUtils.startScmRpcServer(conf,
+        scmServerImpl, address, 10);
+    return scmID;
+  }
+
+  /// CONTAINER OPERATIONS ///
+  public void readChunk(ContainerProtos.WriteChunkRequestProto writeChunk,
+      Pipeline pipeline)  throws Exception {
+    ContainerProtos.ContainerCommandRequestProto readChunkRequest =
+        ContainerTestHelper.getReadChunkRequest(pipeline, writeChunk);
+
+    dispatchRequest(readChunkRequest);
+  }
+
+  public ContainerProtos.WriteChunkRequestProto putBlock(long containerID,
+      Pipeline pipeline) throws Exception {
+    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
+        getWriteChunk(containerID, pipeline);
+    dispatchRequest(writeChunkRequest);
+
+    ContainerProtos.ContainerCommandRequestProto putBlockRequest =
+        ContainerTestHelper.getPutBlockRequest(pipeline,
+            writeChunkRequest.getWriteChunk());
+    dispatchRequest(putBlockRequest);
+
+    return writeChunkRequest.getWriteChunk();
+  }
+
+  public ContainerProtos.ContainerCommandRequestProto getWriteChunk(
+      long containerID, Pipeline pipeline) throws Exception {
+    return ContainerTestHelper.getWriteChunkRequest(pipeline,
+            ContainerTestHelper.getTestBlockID(containerID), 100, null);
+  }
+
+  public Pipeline getPipeline() {
+    return MockPipeline.createPipeline(
+        Collections.singletonList(dsm.getDatanodeDetails()));
+  }
+
+  public long addContainer(Pipeline pipeline)
+      throws Exception {
+    long containerID = random.nextInt(Integer.MAX_VALUE);
+    ContainerProtos.ContainerCommandRequestProto createContainerRequest =
+        ContainerTestHelper.getCreateContainerRequest(containerID, pipeline);
+    dispatchRequest(createContainerRequest);
+
+    return containerID;
+  }
+
+  public void deleteContainer(long containerID, Pipeline pipeline)
+      throws Exception {
+    ContainerProtos.ContainerCommandRequestProto deleteContainerRequest =
+        ContainerTestHelper.getDeleteContainer(pipeline, containerID, true);
+    dispatchRequest(deleteContainerRequest);
+  }
+
+  public void closeContainer(long containerID, Pipeline pipeline)
+      throws Exception {
+    closeContainer(containerID, pipeline, ContainerProtos.Result.SUCCESS);
+  }
+
+  public void closeContainer(long containerID, Pipeline pipeline,
+      ContainerProtos.Result expectedResult) throws Exception {
+    ContainerProtos.ContainerCommandRequestProto closeContainerRequest =
+        ContainerTestHelper.getCloseContainer(pipeline, containerID);
+    dispatchRequest(closeContainerRequest, expectedResult);
+  }
+
+  public void dispatchRequest(
+      ContainerProtos.ContainerCommandRequestProto request) {
+    dispatchRequest(request, ContainerProtos.Result.SUCCESS);
+  }
+
+  public void dispatchRequest(
+      ContainerProtos.ContainerCommandRequestProto request,
+      ContainerProtos.Result expectedResult) {
+    ContainerProtos.ContainerCommandResponseProto response =
+        dsm.getContainer().getDispatcher().dispatch(request, null);
+    Assert.assertEquals(expectedResult, response.getResult());
+  }
+
+  /// VOLUME OPERATIONS ///
+
+  /**
+   * Append a datanode volume to the existing volumes in the configuration.
+   * @return The root directory for the new volume.
+   */
+  public File addHddsVolume() throws IOException {
+    File vol = tempFolder.newFolder(UUID.randomUUID().toString());
+    String[] existingVolumes =
+        conf.getStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY);
+    List<String> allVolumes = new ArrayList<>();
+    if (existingVolumes != null) {
+      allVolumes.addAll(Arrays.asList(existingVolumes));
+    }
+
+    allVolumes.add(vol.getAbsolutePath());
+    conf.setStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY,
+        allVolumes.toArray(new String[0]));
+
+    return vol;
+  }
+
+  /**
+   * Append a db volume to the existing volumes in the configuration.
+   * @return The root directory for the new volume.
+   */
+  public File addDbVolume() throws Exception {
+    File vol = tempFolder.newFolder(UUID.randomUUID().toString());
+    String[] existingVolumes =
+        conf.getStrings(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR);
+    List<String> allVolumes = new ArrayList<>();
+    if (existingVolumes != null) {
+      allVolumes.addAll(Arrays.asList(existingVolumes));
+    }
+
+    allVolumes.add(vol.getAbsolutePath());
+    conf.setStrings(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR,
+        allVolumes.toArray(new String[0]));
+
+    return vol;
+  }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java
index 34fb79a..c5bd439 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java
@@ -192,7 +192,9 @@
     // restarted with SCM HA config and gets a different SCM ID.
     conf.setBoolean(ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY, true);
     changeScmID();
+
     restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true);
+
     // Make sure the existing container can be read.
     readChunk(exportWriteChunk2, pipeline);
 
@@ -344,6 +346,7 @@
     changeScmID();
     // A new volume is added that must be formatted.
     File preFinVolume2 = addVolume();
+
     restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true);
 
     Assert.assertEquals(2,
@@ -378,7 +381,9 @@
     File finVolume = addVolume();
     // Yet another SCM ID is received this time, but it should not matter.
     changeScmID();
+
     restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion(), false);
+
     Assert.assertEquals(3,
         dsm.getContainer().getVolumeSet().getVolumesList().size());
     Assert.assertEquals(0,
diff --git a/hadoop-hdds/docs/content/design/dn-merge-rocksdb.md b/hadoop-hdds/docs/content/design/dn-merge-rocksdb.md
new file mode 100644
index 0000000..4f19588
--- /dev/null
+++ b/hadoop-hdds/docs/content/design/dn-merge-rocksdb.md
@@ -0,0 +1,29 @@
+---
+title: Merge Container RocksDB in DN 
+summary: Use one RocksDB to hold all container metadata on a DN data volume
+date: 2022-05-24
+jira: HDDS-3630
+status: implemented
+
+---
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Abstract
+
+ Use one single RocksDB to hold all container metadata on a DN data volume, replacing the current one RocksDB for each container mode. 
+  
+# Link
+
+https://issues.apache.org/jira/secure/attachment/13044057/Merge%20rocksdb%20in%20datanode%20V5.pdf
\ No newline at end of file
diff --git a/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.md b/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.md
new file mode 100644
index 0000000..92ae64f
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.md
@@ -0,0 +1,70 @@
+---
+title: "Merge Container RocksDB in DN"
+weight: 2
+menu:
+   main:
+      parent: Features
+summary: Introduction to Ozone Datanode Container Schema V3
+---
+<!---
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+In Ozone, user data are separated into blocks and stored in HDDS Containers. Containers are the fundamental replication unit of Ozone/HDDS. Each Container has its metadata and data. Data are saved as files on disk. Metadata is saved in RocksDB.
+
+Currently there will be one RocksDB for each Container on datanode. With user data continously grow, there will be hundreds of thousands of RocksDB instances on one datanode. It's a big challenge to manage this amount of RocksDB instances in one JVM. 
+
+Unlike the current approach, this "Merge Container RocksDB in DN" feature will use only one RocksDB for each data volume, holding all metadata of Containers in this RocksDB. 
+  
+## Configuration
+
+This is mainly a DN feature, which doesn't require much configuration.
+
+Here is a configuration which disable this feature if the current one RocksDB for each container mode is more preferred. Please be noted that once the feature is enabled, it's strongly suggested not to disable it in later. 
+  
+```XML
+<property>
+   <name>hdds.datanode.container.schema.v3.enabled</name>
+   <value>false</value>
+   <description>Disable or enable this feature.</description>
+</property>
+```
+ 
+Without any specific configuration, the single RocksDB will be created under the data volume configured in "hdds.datanode.dir". 
+
+For some advanced cluster admins who have the high performance requirement, he/she can leverage quick storages to save RocksDB. In this case, configure these two properties.  
+
+```XML
+<property>
+   <name>hdds.datanode.container.db.dir</name>
+   <value/>
+   <description>This setting is optional. Specify where the per-disk rocksdb instances will be stored.</description>
+</property>
+<property>
+   <name>hdds.datanode.failed.db.volumes.tolerated</name>
+   <value>-1</value>
+   <description>The number of db volumes that are allowed to fail before a datanode stops offering service.
+   Default -1 means unlimited, but we should have at least one good volume left.</description>
+</property>
+```
+
+### Backward compatibility 
+
+Existing containers each has one RocksDB for them will be still accessible after this feature is enabled. All container data will co-exist in an existing Ozone cluster.
+
+## References
+
+ * [Design doc]({{< ref "design/dn-merge-rocksdb.md">}})
\ No newline at end of file
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java
index f64db4d..80970ff 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java
@@ -92,6 +92,7 @@
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_INFO_WAIT_DURATION;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_INFO_WAIT_DURATION_DEFAULT;
 import static org.apache.hadoop.hdds.server.ServerUtils.sanitizeUserArgs;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR;
 
 import org.rocksdb.RocksDBException;
 import org.slf4j.Logger;
@@ -404,6 +405,12 @@
     return rawLocations;
   }
 
+  public static Collection<String> getDatanodeDbDirs(
+      ConfigurationSource conf) {
+    // No fallback here, since this config is optional.
+    return conf.getTrimmedStringCollection(HDDS_DATANODE_CONTAINER_DB_DIR);
+  }
+
   /**
    * Get the path for datanode id file.
    *
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MetadataKeyFilters.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MetadataKeyFilters.java
index e3b91ba..31660c0 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MetadataKeyFilters.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MetadataKeyFilters.java
@@ -32,6 +32,7 @@
 public final class MetadataKeyFilters {
   private MetadataKeyFilters() { }
 
+  @Deprecated
   public static KeyPrefixFilter getDeletingKeyFilter() {
     return new MetadataKeyFilters.KeyPrefixFilter()
             .addFilter(OzoneConsts.DELETING_KEY_PREFIX);
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBColumnFamilyDefinition.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBColumnFamilyDefinition.java
index e1c4163..8c66bc6 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBColumnFamilyDefinition.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBColumnFamilyDefinition.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hdds.utils.db;
 
+import org.rocksdb.ColumnFamilyOptions;
+
 import java.io.IOException;
 
 /**
@@ -38,6 +40,8 @@
 
   private final Codec<VALUE> valueCodec;
 
+  private ColumnFamilyOptions cfOptions;
+
   public DBColumnFamilyDefinition(
       String tableName,
       Class<KEY> keyType,
@@ -49,6 +53,7 @@
     this.keyCodec = keyCodec;
     this.valueType = valueType;
     this.valueCodec = valueCodec;
+    this.cfOptions = null;
   }
 
   public Table<KEY, VALUE> getTable(DBStore db) throws IOException {
@@ -78,4 +83,12 @@
   public Codec<VALUE> getValueCodec() {
     return valueCodec;
   }
+
+  public ColumnFamilyOptions getCfOptions() {
+    return this.cfOptions;
+  }
+
+  public void setCfOptions(ColumnFamilyOptions cfOptions) {
+    this.cfOptions = cfOptions;
+  }
 }
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java
index 8b07003..21a6bec 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java
@@ -156,7 +156,7 @@
     for (DBColumnFamilyDefinition columnFamily :
         definition.getColumnFamilies()) {
 
-      addTable(columnFamily.getName());
+      addTable(columnFamily.getName(), columnFamily.getCfOptions());
       addCodec(columnFamily.getKeyType(), columnFamily.getKeyCodec());
       addCodec(columnFamily.getValueType(), columnFamily.getValueCodec());
     }
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DumpFileLoader.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DumpFileLoader.java
new file mode 100644
index 0000000..16655cc
--- /dev/null
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DumpFileLoader.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.utils.db;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Interface for loading data from a dump file.
+ */
+public interface DumpFileLoader extends Closeable {
+
+  /**
+   * Load key value pairs from an external dump file.
+   */
+  void load(File externalFile) throws IOException;
+
+  /**
+   * Close this file loader.
+   */
+  void close();
+}
\ No newline at end of file
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DumpFileWriter.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DumpFileWriter.java
new file mode 100644
index 0000000..f26ba50
--- /dev/null
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DumpFileWriter.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.utils.db;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Interface for write data into a dump file.
+ */
+public interface DumpFileWriter extends Closeable {
+  /**
+   * Open an external file for dump.
+   * @param externalFile
+   */
+  void open(File externalFile) throws IOException;
+
+  /**
+   * Put a key value pair into the file.
+   * @param key
+   * @param value
+   */
+  void put(byte[] key, byte[] value) throws IOException;
+
+  /**
+   * Finish dumping.
+   */
+  @Override
+  void close() throws IOException;
+}
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/FixedLengthStringCodec.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/FixedLengthStringCodec.java
new file mode 100644
index 0000000..a493e6a
--- /dev/null
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/FixedLengthStringCodec.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.hdds.utils.db;
+
+import java.io.IOException;
+
+/**
+ * Codec to convert a prefixed String to/from byte array.
+ * The prefix has to be of fixed-length.
+ */
+public class FixedLengthStringCodec implements Codec<String> {
+  @Override
+  public byte[] toPersistedFormat(String object) throws IOException {
+    if (object != null) {
+      return FixedLengthStringUtils.string2Bytes(object);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public String fromPersistedFormat(byte[] rawData) throws IOException {
+    if (rawData != null) {
+      return FixedLengthStringUtils.bytes2String(rawData);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public String copyObject(String object) {
+    return object;
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/FixedLengthStringUtils.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/FixedLengthStringUtils.java
new file mode 100644
index 0000000..ce2c59a
--- /dev/null
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/FixedLengthStringUtils.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.utils.db;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * String utility class for conversion between byte[] and string
+ * which requires string to be of non-variable-length encoding(e.g. ASCII).
+ * This is different from StringUtils which uses UTF-8 encoding which is
+ * a variable-length encoding style.
+ * This is mainly for FixedLengthStringCodec which requires a fixed-length
+ * prefix.
+ */
+public final class FixedLengthStringUtils {
+
+  private FixedLengthStringUtils() {
+  }
+
+  // An ASCII extension: https://en.wikipedia.org/wiki/ISO/IEC_8859-1
+  // Each character is encoded as a single eight-bit code value.
+  private static final String ASCII_CSN = StandardCharsets.ISO_8859_1.name();
+
+  public static String bytes2String(byte[] bytes) {
+    try {
+      return new String(bytes, 0, bytes.length, ASCII_CSN);
+    } catch (UnsupportedEncodingException e) {
+      throw new IllegalArgumentException(
+          "ISO_8859_1 encoding is not supported", e);
+    }
+  }
+
+  public static byte[] string2Bytes(String str) {
+    try {
+      return str.getBytes(ASCII_CSN);
+    } catch (UnsupportedEncodingException e) {
+      throw new IllegalArgumentException(
+          "ISO_8859_1 decoding is not supported", e);
+    }
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBSstFileLoader.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBSstFileLoader.java
new file mode 100644
index 0000000..e98a29f
--- /dev/null
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBSstFileLoader.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.utils.db;
+
+import org.rocksdb.IngestExternalFileOptions;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+
+import org.apache.hadoop.hdds.utils.db.RocksDatabase.ColumnFamily;
+
+/**
+ * DumpFileLoader using rocksdb sst files.
+ */
+public class RDBSstFileLoader implements DumpFileLoader, Closeable {
+
+  private final RocksDatabase db;
+  private final ColumnFamily family;
+
+  public RDBSstFileLoader(RocksDatabase db, ColumnFamily cf) {
+    this.db = db;
+    this.family = cf;
+  }
+
+  @Override
+  public void load(File externalFile) throws IOException {
+    // Ingest an empty sst file results in exception.
+    if (externalFile.length() == 0) {
+      return;
+    }
+    IngestExternalFileOptions ingestOptions = new IngestExternalFileOptions()
+        .setIngestBehind(false);
+    try {
+      db.ingestExternalFile(family,
+          Collections.singletonList(externalFile.getAbsolutePath()),
+          ingestOptions);
+    } finally {
+      ingestOptions.close();
+    }
+  }
+
+  @Override
+  public void close() {
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBSstFileWriter.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBSstFileWriter.java
new file mode 100644
index 0000000..367c235
--- /dev/null
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBSstFileWriter.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.utils.db;
+
+import org.rocksdb.EnvOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.SstFileWriter;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.hdds.utils.HddsServerUtil.toIOException;
+
+/**
+ * DumpFileWriter using rocksdb sst files.
+ */
+public class RDBSstFileWriter implements DumpFileWriter, Closeable {
+
+  private SstFileWriter sstFileWriter;
+  private File sstFile;
+  private AtomicLong keyCounter;
+
+  public RDBSstFileWriter() {
+    EnvOptions envOptions = new EnvOptions();
+    this.sstFileWriter = new SstFileWriter(envOptions, new Options());
+    this.keyCounter = new AtomicLong(0);
+  }
+
+  @Override
+  public void open(File externalFile) throws IOException {
+    this.sstFile = externalFile;
+    try {
+      // Here will create a new sst file each time, not append to existing
+      sstFileWriter.open(sstFile.getAbsolutePath());
+    } catch (RocksDBException e) {
+      closeOnFailure();
+      throw toIOException("Failed to open external file for dump "
+          + sstFile.getAbsolutePath(), e);
+    }
+  }
+
+  @Override
+  public void put(byte[] key, byte[] value) throws IOException {
+    try {
+      sstFileWriter.put(key, value);
+      keyCounter.incrementAndGet();
+    } catch (RocksDBException e) {
+      closeOnFailure();
+      throw toIOException("Failed to put kv into dump file "
+          + sstFile.getAbsolutePath(), e);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (sstFileWriter != null) {
+      try {
+        // We should check for empty sst file, or we'll get exception.
+        if (keyCounter.get() > 0) {
+          sstFileWriter.finish();
+        }
+      } catch (RocksDBException e) {
+        throw toIOException("Failed to finish dumping into file "
+            + sstFile.getAbsolutePath(), e);
+      } finally {
+        sstFileWriter.close();
+        sstFileWriter = null;
+      }
+
+      keyCounter.set(0);
+    }
+  }
+
+  private void closeOnFailure() {
+    if (sstFileWriter != null) {
+      sstFileWriter.close();
+      sstFileWriter = null;
+    }
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreIterator.java
index dbe5625..8ef2b1d 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreIterator.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreIterator.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hdds.utils.db;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.NoSuchElementException;
 import java.util.function.Consumer;
 
@@ -39,15 +40,26 @@
   private final RocksIterator rocksDBIterator;
   private RDBTable rocksDBTable;
   private ByteArrayKeyValue currentEntry;
+  // This is for schemas that use a fixed-length
+  // prefix for each key.
+  private byte[] prefix;
 
   public RDBStoreIterator(RocksIterator iterator) {
-    this.rocksDBIterator = iterator;
-    seekToFirst();
+    this(iterator, null);
   }
 
   public RDBStoreIterator(RocksIterator iterator, RDBTable table) {
-    this(iterator);
+    this(iterator, table, null);
+  }
+
+  public RDBStoreIterator(RocksIterator iterator, RDBTable table,
+      byte[] prefix) {
+    this.rocksDBIterator = iterator;
     this.rocksDBTable = table;
+    if (prefix != null) {
+      this.prefix = Arrays.copyOf(prefix, prefix.length);
+    }
+    seekToFirst();
   }
 
   @Override
@@ -69,7 +81,8 @@
 
   @Override
   public boolean hasNext() {
-    return rocksDBIterator.isValid();
+    return rocksDBIterator.isValid() &&
+        (prefix == null || startsWith(prefix, rocksDBIterator.key()));
   }
 
   @Override
@@ -84,13 +97,21 @@
 
   @Override
   public void seekToFirst() {
-    rocksDBIterator.seekToFirst();
+    if (prefix == null) {
+      rocksDBIterator.seekToFirst();
+    } else {
+      rocksDBIterator.seek(prefix);
+    }
     setCurrentEntry();
   }
 
   @Override
   public void seekToLast() {
-    rocksDBIterator.seekToLast();
+    if (prefix == null) {
+      rocksDBIterator.seekToLast();
+    } else {
+      throw new UnsupportedOperationException("seekToLast");
+    }
     setCurrentEntry();
   }
 
@@ -117,4 +138,25 @@
   public void close() throws IOException {
     rocksDBIterator.close();
   }
+
+  private static boolean startsWith(byte[] prefix, byte[] value) {
+    if (prefix == null) {
+      return true;
+    }
+    if (value == null) {
+      return false;
+    }
+
+    int length = prefix.length;
+    if (value.length < length) {
+      return false;
+    }
+
+    for (int i = 0; i < length; i++) {
+      if (value[i] != prefix[i]) {
+        return false;
+      }
+    }
+    return true;
+  }
 }
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java
index f6321fc..d2bb381 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hdds.utils.db;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import java.util.ArrayList;
@@ -27,8 +28,6 @@
 
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
 import org.apache.hadoop.hdds.utils.db.RocksDatabase.ColumnFamily;
-
-import org.rocksdb.RocksIterator;
 import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -163,6 +162,12 @@
   }
 
   @Override
+  public TableIterator<byte[], ByteArrayKeyValue> iterator(byte[] prefix) {
+    return new RDBStoreIterator(db.newIterator(family, false), this,
+        prefix);
+  }
+
+  @Override
   public String getName() throws IOException {
     return family.getName();
   }
@@ -179,58 +184,87 @@
 
   @Override
   public List<ByteArrayKeyValue> getRangeKVs(byte[] startKey,
-      int count, MetadataKeyFilters.MetadataKeyFilter... filters)
+      int count, byte[] prefix,
+      MetadataKeyFilters.MetadataKeyFilter... filters)
       throws IOException, IllegalArgumentException {
-    return getRangeKVs(startKey, count, false, filters);
+    return getRangeKVs(startKey, count, false, prefix, filters);
   }
 
   @Override
   public List<ByteArrayKeyValue> getSequentialRangeKVs(byte[] startKey,
-      int count, MetadataKeyFilters.MetadataKeyFilter... filters)
+      int count, byte[] prefix,
+      MetadataKeyFilters.MetadataKeyFilter... filters)
       throws IOException, IllegalArgumentException {
-    return getRangeKVs(startKey, count, true, filters);
+    return getRangeKVs(startKey, count, true, prefix, filters);
+  }
+
+  @Override
+  public void deleteBatchWithPrefix(BatchOperation batch, byte[] prefix)
+      throws IOException {
+    try (TableIterator<byte[], ByteArrayKeyValue> iter = iterator(prefix)) {
+      while (iter.hasNext()) {
+        deleteWithBatch(batch, iter.next().getValue());
+      }
+    }
+  }
+
+  @Override
+  public void dumpToFileWithPrefix(File externalFile, byte[] prefix)
+      throws IOException {
+    try (TableIterator<byte[], ByteArrayKeyValue> iter = iterator(prefix);
+         DumpFileWriter fileWriter = new RDBSstFileWriter()) {
+      fileWriter.open(externalFile);
+      while (iter.hasNext()) {
+        ByteArrayKeyValue entry = iter.next();
+        fileWriter.put(entry.getKey(), entry.getValue());
+      }
+    }
+  }
+
+  @Override
+  public void loadFromFile(File externalFile) throws IOException {
+    try (DumpFileLoader fileLoader = new RDBSstFileLoader(db, family)) {
+      fileLoader.load(externalFile);
+    }
   }
 
   private List<ByteArrayKeyValue> getRangeKVs(byte[] startKey,
-      int count, boolean sequential,
+      int count, boolean sequential, byte[] prefix,
       MetadataKeyFilters.MetadataKeyFilter... filters)
       throws IOException, IllegalArgumentException {
     List<ByteArrayKeyValue> result = new ArrayList<>();
     long start = System.currentTimeMillis();
+
     if (count < 0) {
       throw new IllegalArgumentException(
             "Invalid count given " + count + ", count must be greater than 0");
     }
-    try (RocksIterator it = db.newIterator(family)) {
+    try (TableIterator<byte[], ByteArrayKeyValue> it = iterator(prefix)) {
       if (startKey == null) {
         it.seekToFirst();
       } else {
-        if (get(startKey) == null) {
+        if ((prefix == null || startKey.length > prefix.length)
+            && get(startKey) == null) {
           // Key not found, return empty list
           return result;
         }
         it.seek(startKey);
       }
-      while (it.isValid() && result.size() < count) {
-        byte[] currentKey = it.key();
-        byte[] currentValue = it.value();
 
-        it.prev();
-        final byte[] prevKey = it.isValid() ? it.key() : null;
-
-        it.seek(currentKey);
-        it.next();
-        final byte[] nextKey = it.isValid() ? it.key() : null;
+      while (it.hasNext() && result.size() < count) {
+        ByteArrayKeyValue currentEntry = it.next();
+        byte[] currentKey = currentEntry.getKey();
 
         if (filters == null) {
-          result.add(ByteArrayKeyValue
-                  .create(currentKey, currentValue));
+          result.add(currentEntry);
         } else {
+          // NOTE: the preKey and nextKey are never checked
+          // in all existing underlying filters, so they could
+          // be safely as null here.
           if (Arrays.stream(filters)
-                  .allMatch(entry -> entry.filterKey(prevKey,
-                          currentKey, nextKey))) {
-            result.add(ByteArrayKeyValue
-                    .create(currentKey, currentValue));
+                  .allMatch(entry -> entry.filterKey(null,
+                          currentKey, null))) {
+            result.add(currentEntry);
           } else {
             if (result.size() > 0 && sequential) {
               // if the caller asks for a sequential range of results,
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java
index 2d358e1..50b3723 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java
@@ -24,6 +24,7 @@
 import org.rocksdb.DBOptions;
 import org.rocksdb.FlushOptions;
 import org.rocksdb.Holder;
+import org.rocksdb.IngestExternalFileOptions;
 import org.rocksdb.Options;
 import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
@@ -227,7 +228,7 @@
       return codec.fromPersistedFormat(nameBytes);
     }
 
-    private ColumnFamilyHandle getHandle() {
+    protected ColumnFamilyHandle getHandle() {
       return handle;
     }
 
@@ -302,6 +303,19 @@
     }
   }
 
+  public void ingestExternalFile(ColumnFamily family, List<String> files,
+      IngestExternalFileOptions ingestOptions) throws IOException {
+    try {
+      db.ingestExternalFile(family.getHandle(), files, ingestOptions);
+    } catch (RocksDBException e) {
+      closeOnError(e);
+      String msg = "Failed to ingest external files " +
+          files.stream().collect(Collectors.joining(", ")) + " of " +
+          family.getName();
+      throw toIOException(this, msg, e);
+    }
+  }
+
   public void put(ColumnFamily family, byte[] key, byte[] value)
       throws IOException {
     try {
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java
index 554af5a..3202431 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hdds.utils.db;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
@@ -154,6 +155,14 @@
   TableIterator<KEY, ? extends KeyValue<KEY, VALUE>> iterator();
 
   /**
+   * Returns a prefixed iterator for this metadata store.
+   * @param prefix
+   * @return
+   */
+  TableIterator<KEY, ? extends KeyValue<KEY, VALUE>> iterator(KEY prefix)
+      throws IOException;
+
+  /**
    * Returns the Name of this Table.
    * @return - Table Name.
    * @throws IOException on failure.
@@ -228,6 +237,7 @@
    *
    * @param startKey a start key.
    * @param count max number of entries to return.
+   * @param prefix fixed key schema specific prefix
    * @param filters customized one or more
    * {@link MetadataKeyFilters.MetadataKeyFilter}.
    * @return a list of entries found in the database or an empty list if the
@@ -236,7 +246,8 @@
    * @throws IllegalArgumentException if count is less than 0.
    */
   List<? extends KeyValue<KEY, VALUE>> getRangeKVs(KEY startKey,
-          int count, MetadataKeyFilters.MetadataKeyFilter... filters)
+          int count, KEY prefix,
+          MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException;
 
   /**
@@ -249,6 +260,7 @@
    *
    * @param startKey a start key.
    * @param count max number of entries to return.
+   * @param prefix fixed key schema specific prefix
    * @param filters customized one or more
    * {@link MetadataKeyFilters.MetadataKeyFilter}.
    * @return a list of entries found in the database.
@@ -256,10 +268,37 @@
    * @throws IllegalArgumentException
    */
   List<? extends KeyValue<KEY, VALUE>> getSequentialRangeKVs(KEY startKey,
-          int count, MetadataKeyFilters.MetadataKeyFilter... filters)
+          int count, KEY prefix,
+          MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException;
 
   /**
+   * Deletes all keys with the specified prefix from the metadata store
+   * as part of a batch operation.
+   * @param batch
+   * @param prefix
+   * @return
+   */
+  void deleteBatchWithPrefix(BatchOperation batch, KEY prefix)
+      throws IOException;
+
+  /**
+   * Dump all key value pairs with a prefix into an external file.
+   * @param externalFile
+   * @param prefix
+   * @throws IOException
+   */
+  void dumpToFileWithPrefix(File externalFile, KEY prefix) throws IOException;
+
+  /**
+   * Load key value pairs from an external file created by
+   * dumpToFileWithPrefix.
+   * @param externalFile
+   * @throws IOException
+   */
+  void loadFromFile(File externalFile) throws IOException;
+
+  /**
    * Class used to represent the key and value pair of a db entry.
    */
   interface KeyValue<KEY, VALUE> {
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java
index c7f6196..c43855a 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hdds.utils.db;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
@@ -281,6 +282,14 @@
   }
 
   @Override
+  public TableIterator<KEY, TypedKeyValue> iterator(KEY prefix)
+      throws IOException {
+    TableIterator<byte[], ? extends KeyValue<byte[], byte[]>> iterator =
+        rawTable.iterator(codecRegistry.asRawData(prefix));
+    return new TypedTableIterator(iterator, keyType, valueType);
+  }
+
+  @Override
   public String getName() throws IOException {
     return rawTable.getName();
   }
@@ -315,19 +324,23 @@
 
   @Override
   public List<TypedKeyValue> getRangeKVs(
-          KEY startKey, int count,
+          KEY startKey, int count, KEY prefix,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException {
 
     // A null start key means to start from the beginning of the table.
     // Cannot convert a null key to bytes.
     byte[] startKeyBytes = null;
+    byte[] prefixBytes = null;
     if (startKey != null) {
       startKeyBytes = codecRegistry.asRawData(startKey);
     }
+    if (prefix != null) {
+      prefixBytes = codecRegistry.asRawData(prefix);
+    }
 
     List<? extends KeyValue<byte[], byte[]>> rangeKVBytes =
-            rawTable.getRangeKVs(startKeyBytes, count, filters);
+        rawTable.getRangeKVs(startKeyBytes, count, prefixBytes, filters);
 
     List<TypedKeyValue> rangeKVs = new ArrayList<>();
     rangeKVBytes.forEach(byteKV -> rangeKVs.add(new TypedKeyValue(byteKV)));
@@ -337,19 +350,24 @@
 
   @Override
   public List<TypedKeyValue> getSequentialRangeKVs(
-          KEY startKey, int count,
+          KEY startKey, int count, KEY prefix,
           MetadataKeyFilters.MetadataKeyFilter... filters)
           throws IOException, IllegalArgumentException {
 
     // A null start key means to start from the beginning of the table.
     // Cannot convert a null key to bytes.
     byte[] startKeyBytes = null;
+    byte[] prefixBytes = null;
     if (startKey != null) {
       startKeyBytes = codecRegistry.asRawData(startKey);
     }
+    if (prefix != null) {
+      prefixBytes = codecRegistry.asRawData(prefix);
+    }
 
     List<? extends KeyValue<byte[], byte[]>> rangeKVBytes =
-            rawTable.getSequentialRangeKVs(startKeyBytes, count, filters);
+        rawTable.getSequentialRangeKVs(startKeyBytes, count,
+            prefixBytes, filters);
 
     List<TypedKeyValue> rangeKVs = new ArrayList<>();
     rangeKVBytes.forEach(byteKV -> rangeKVs.add(new TypedKeyValue(byteKV)));
@@ -358,6 +376,24 @@
   }
 
   @Override
+  public void deleteBatchWithPrefix(BatchOperation batch, KEY prefix)
+      throws IOException {
+    rawTable.deleteBatchWithPrefix(batch, codecRegistry.asRawData(prefix));
+  }
+
+  @Override
+  public void dumpToFileWithPrefix(File externalFile, KEY prefix)
+      throws IOException {
+    rawTable.dumpToFileWithPrefix(externalFile,
+        codecRegistry.asRawData(prefix));
+  }
+
+  @Override
+  public void loadFromFile(File externalFile) throws IOException {
+    rawTable.loadFromFile(externalFile);
+  }
+
+  @Override
   public void cleanupCache(List<Long> epochs) {
     cache.cleanup(epochs);
   }
diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestDBStoreBuilder.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestDBStoreBuilder.java
index eca86d4..8aaead3 100644
--- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestDBStoreBuilder.java
+++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestDBStoreBuilder.java
@@ -20,16 +20,22 @@
 package org.apache.hadoop.hdds.utils.db;
 
 import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
+import org.junit.Assert;
 import org.rocksdb.ColumnFamilyOptions;
 
+import java.io.File;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collection;
 
 /**
  * Tests RDBStore creation.
@@ -152,5 +158,78 @@
     }
   }
 
+  @Test
+  public void builderWithColumnFamilyOptions(@TempDir Path tempDir)
+      throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempDir.toString());
+    File newFolder = new File(tempDir.toString() + "/newFolder");
 
+    if (!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+
+    String sampleTableName = "sampleTable";
+    final DBDefinition sampleDB = new DBDefinition() {
+
+      private final DBColumnFamilyDefinition<String, Long> sampleTable =
+          new DBColumnFamilyDefinition<>(sampleTableName,
+              String.class, new StringCodec(), Long.class, new LongCodec());
+      {
+        ColumnFamilyOptions cfOptions = new ColumnFamilyOptions();
+        // reverse the default option for check
+        cfOptions.setForceConsistencyChecks(
+            !cfOptions.forceConsistencyChecks());
+        sampleTable.setCfOptions(cfOptions);
+      }
+
+      @Override
+      public String getName() {
+        return "sampleDB";
+      }
+
+      @Override
+      public String getLocationConfigKey() {
+        return null;
+      }
+
+      @Override
+      public DBColumnFamilyDefinition[] getColumnFamilies() {
+        return new DBColumnFamilyDefinition[]{sampleTable};
+      }
+
+      @Override
+      public File getDBLocation(ConfigurationSource conf) {
+        return null;
+      }
+    };
+
+    try (DBStore dbStore = DBStoreBuilder.newBuilder(conf, sampleDB)
+        .setName("SampleStore").setPath(newFolder.toPath()).build()) {
+      Assert.assertTrue(dbStore instanceof RDBStore);
+
+      RDBStore rdbStore = (RDBStore) dbStore;
+      Collection<RocksDatabase.ColumnFamily> cfFamilies =
+          rdbStore.getColumnFamilies();
+
+      // we also have the default column family, so there are 2
+      Assert.assertEquals(2, cfFamilies.size());
+
+      boolean checked = false;
+      for (RocksDatabase.ColumnFamily cfFamily : cfFamilies) {
+        if (Arrays.equals(cfFamily.getHandle().getName(),
+            sampleTableName.getBytes(StandardCharsets.UTF_8))) {
+          // get the default value
+          boolean defaultValue = new ColumnFamilyOptions()
+              .forceConsistencyChecks();
+
+          // the value should be different from the default value
+          Assert.assertNotEquals(cfFamily.getHandle().getDescriptor()
+              .getOptions().forceConsistencyChecks(), defaultValue);
+          checked = true;
+        }
+      }
+      Assert.assertTrue(checked);
+    }
+  }
 }
\ No newline at end of file
diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestFixedLengthStringUtils.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestFixedLengthStringUtils.java
new file mode 100644
index 0000000..d2ad45b
--- /dev/null
+++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestFixedLengthStringUtils.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.utils.db;
+
+import com.google.common.primitives.Longs;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for class FixedLengthStringUtils.
+ */
+public class TestFixedLengthStringUtils {
+
+  @Test
+  public void testStringEncodeAndDecode() {
+    long[] testContainerIDs = {
+        0L, 1L, 2L, 12345L,
+        Long.MAX_VALUE / 2, Long.MAX_VALUE - 1, Long.MAX_VALUE
+    };
+
+    for (long containerID : testContainerIDs) {
+      String containerPrefix = FixedLengthStringUtils.bytes2String(
+          Longs.toByteArray(containerID));
+      long decodedContainerID = Longs.fromByteArray(
+          FixedLengthStringUtils.string2Bytes(containerPrefix));
+      assertEquals(containerID, decodedContainerID);
+    }
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreIterator.java
index 243848e..5eda82c 100644
--- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreIterator.java
+++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreIterator.java
@@ -24,6 +24,8 @@
 import org.mockito.InOrder;
 import org.rocksdb.RocksIterator;
 
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.NoSuchElementException;
 import java.util.function.Consumer;
 
@@ -31,7 +33,9 @@
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.clearInvocations;
 import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
@@ -236,4 +240,54 @@
 
     verify(rocksDBIteratorMock, times(1)).close();
   }
+
+  @Test
+  public void testNullPrefixedIterator() throws IOException {
+    RDBStoreIterator iter = new RDBStoreIterator(rocksDBIteratorMock,
+        rocksTableMock, null);
+    verify(rocksDBIteratorMock, times(1)).seekToFirst();
+    clearInvocations(rocksDBIteratorMock);
+
+    iter.seekToFirst();
+    verify(rocksDBIteratorMock, times(1)).seekToFirst();
+    clearInvocations(rocksDBIteratorMock);
+
+    when(rocksDBIteratorMock.isValid()).thenReturn(true);
+    assertTrue(iter.hasNext());
+    verify(rocksDBIteratorMock, times(1)).isValid();
+    verify(rocksDBIteratorMock, times(0)).key();
+
+    iter.seekToLast();
+    verify(rocksDBIteratorMock, times(1)).seekToLast();
+
+    iter.close();
+  }
+
+  @Test
+  public void testNormalPrefixedIterator() throws IOException {
+    byte[] testPrefix = "sample".getBytes(StandardCharsets.UTF_8);
+    RDBStoreIterator iter = new RDBStoreIterator(rocksDBIteratorMock,
+        rocksTableMock, testPrefix);
+    verify(rocksDBIteratorMock, times(1)).seek(testPrefix);
+    clearInvocations(rocksDBIteratorMock);
+
+    iter.seekToFirst();
+    verify(rocksDBIteratorMock, times(1)).seek(testPrefix);
+    clearInvocations(rocksDBIteratorMock);
+
+    when(rocksDBIteratorMock.isValid()).thenReturn(true);
+    when(rocksDBIteratorMock.key()).thenReturn(testPrefix);
+    assertTrue(iter.hasNext());
+    verify(rocksDBIteratorMock, times(1)).isValid();
+    verify(rocksDBIteratorMock, times(1)).key();
+
+    try {
+      iter.seekToLast();
+      fail("Prefixed iterator does not support seekToLast");
+    } catch (Exception e) {
+      assertTrue(e instanceof UnsupportedOperationException);
+    }
+
+    iter.close();
+  }
 }
diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java
index dfb6e6f..d70a496 100644
--- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java
+++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java
@@ -22,10 +22,13 @@
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hdds.StringUtils;
@@ -36,6 +39,9 @@
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
+import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
+import org.junit.Assert;
+import org.junit.Rule;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 import org.rocksdb.RocksDB;
@@ -53,6 +59,13 @@
           "Fourth", "Fifth",
           "Sixth", "Seventh",
           "Eighth");
+  private final List<String> prefixedFamilies = Arrays.asList(
+      "PrefixFirst",
+      "PrefixTwo", "PrefixThree",
+      "PrefixFour", "PrefixFifth"
+  );
+  private static final int PREFIX_LENGTH = 9;
+  @Rule
   private RDBStore rdbStore = null;
   private DBOptions options = null;
   private static byte[][] bytesOf;
@@ -90,6 +103,13 @@
       TableConfig newConfig = new TableConfig(name, new ColumnFamilyOptions());
       configSet.add(newConfig);
     }
+    for (String name : prefixedFamilies) {
+      ColumnFamilyOptions cfOptions = new ColumnFamilyOptions();
+      cfOptions.useFixedLengthPrefixExtractor(PREFIX_LENGTH);
+
+      TableConfig newConfig = new TableConfig(name, cfOptions);
+      configSet.add(newConfig);
+    }
     rdbStore = new RDBStore(tempDir, options, configSet);
   }
 
@@ -430,4 +450,210 @@
       testTable.put(key, value);
     }
   }
+
+  @Test
+  public void testPrefixedIterator() throws Exception {
+    int containerCount = 3;
+    int blockCount = 5;
+    List<String> testPrefixes = generatePrefixes(containerCount);
+    List<Map<String, String>> testData = generateKVs(testPrefixes, blockCount);
+
+    try (Table<byte[], byte[]> testTable = rdbStore.getTable("PrefixFirst")) {
+      // write data
+      populatePrefixedTable(testTable, testData);
+
+      // iterator should seek to right pos in the middle
+      byte[] samplePrefix = testPrefixes.get(2).getBytes(
+          StandardCharsets.UTF_8);
+      try (TableIterator<byte[],
+          ? extends Table.KeyValue<byte[], byte[]>> iter = testTable.iterator(
+              samplePrefix)) {
+        int keyCount = 0;
+        while (iter.hasNext()) {
+          // iterator should only meet keys with samplePrefix
+          Assert.assertTrue(Arrays.equals(
+              Arrays.copyOf(iter.next().getKey(), PREFIX_LENGTH),
+              samplePrefix));
+          keyCount++;
+        }
+
+        // iterator should end at right pos
+        Assert.assertEquals(blockCount, keyCount);
+
+        // iterator should be able to seekToFirst
+        iter.seekToFirst();
+        Assert.assertTrue(iter.hasNext());
+        Assert.assertTrue(Arrays.equals(
+            Arrays.copyOf(iter.next().getKey(), PREFIX_LENGTH),
+            samplePrefix));
+      }
+    }
+  }
+
+  @Test
+  public void testPrefixedRangeKVs() throws Exception {
+    int containerCount = 3;
+    int blockCount = 5;
+    List<String> testPrefixes = generatePrefixes(containerCount);
+    List<Map<String, String>> testData = generateKVs(testPrefixes, blockCount);
+
+    try (Table<byte[], byte[]> testTable = rdbStore.getTable("PrefixFirst")) {
+
+      // write data
+      populatePrefixedTable(testTable, testData);
+
+      byte[] samplePrefix = testPrefixes.get(2).getBytes(
+          StandardCharsets.UTF_8);
+
+      // test start at first
+      byte[] startKey = samplePrefix;
+      List<? extends Table.KeyValue<byte[], byte[]>> rangeKVs = testTable
+          .getRangeKVs(startKey, 3, samplePrefix);
+      Assert.assertEquals(3, rangeKVs.size());
+
+      // test start with a middle key
+      startKey = StringUtils.string2Bytes(
+          StringUtils.bytes2String(samplePrefix) + "3");
+      rangeKVs = testTable.getRangeKVs(startKey, blockCount, samplePrefix);
+      Assert.assertEquals(2, rangeKVs.size());
+
+      // test with a filter
+      MetadataKeyFilters.KeyPrefixFilter filter1 = new MetadataKeyFilters
+          .KeyPrefixFilter()
+          .addFilter(StringUtils.bytes2String(samplePrefix) + "1");
+      startKey = StringUtils.string2Bytes(
+          StringUtils.bytes2String(samplePrefix));
+      rangeKVs = testTable.getRangeKVs(startKey, blockCount,
+          samplePrefix, filter1);
+      Assert.assertEquals(1, rangeKVs.size());
+
+      // test start with a non-exist key
+      startKey = StringUtils.string2Bytes(
+          StringUtils.bytes2String(samplePrefix) + 123);
+      rangeKVs = testTable.getRangeKVs(startKey, 10, samplePrefix);
+      Assert.assertEquals(0, rangeKVs.size());
+    }
+  }
+
+  @Test
+  public void testDumpAndLoadBasic(@TempDir Path tempDir) throws Exception {
+    int containerCount = 3;
+    int blockCount = 5;
+    List<String> testPrefixes = generatePrefixes(containerCount);
+    List<Map<String, String>> testData = generateKVs(testPrefixes, blockCount);
+    File dumpFile = new File(tempDir.toString() + "/PrefixTwo.dump");
+    byte[] samplePrefix = testPrefixes.get(2).getBytes(StandardCharsets.UTF_8);
+
+    try (Table<byte[], byte[]> testTable1 = rdbStore.getTable("PrefixTwo")) {
+      // write data
+      populatePrefixedTable(testTable1, testData);
+
+      // dump to external file
+      testTable1.dumpToFileWithPrefix(dumpFile, samplePrefix);
+
+      // check dump file exist
+      Assert.assertTrue(dumpFile.exists());
+      Assert.assertTrue(dumpFile.length() != 0);
+    }
+
+    // load dump file into another table
+    try (Table<byte[], byte[]> testTable2 = rdbStore.getTable("PrefixThree")) {
+      testTable2.loadFromFile(dumpFile);
+
+      // check loaded keys
+      try (TableIterator<byte[],
+          ? extends Table.KeyValue<byte[], byte[]>> iter = testTable2.iterator(
+          samplePrefix)) {
+        int keyCount = 0;
+        while (iter.hasNext()) {
+          // check prefix
+          Assert.assertTrue(Arrays.equals(
+              Arrays.copyOf(iter.next().getKey(), PREFIX_LENGTH),
+              samplePrefix));
+          keyCount++;
+        }
+
+        // check block count
+        Assert.assertEquals(blockCount, keyCount);
+      }
+    }
+  }
+
+  @Test
+  public void testDumpAndLoadEmpty(@TempDir Path tempDir) throws Exception {
+    int containerCount = 3;
+    List<String> testPrefixes = generatePrefixes(containerCount);
+
+    File dumpFile = new File(tempDir.toString() + "/PrefixFour.dump");
+    byte[] samplePrefix = testPrefixes.get(2).getBytes(StandardCharsets.UTF_8);
+
+    try (Table<byte[], byte[]> testTable1 = rdbStore.getTable("PrefixFour")) {
+      // no data
+
+      // dump to external file
+      testTable1.dumpToFileWithPrefix(dumpFile, samplePrefix);
+
+      // check dump file exist
+      Assert.assertTrue(dumpFile.exists());
+      // empty dump file
+      Assert.assertTrue(dumpFile.length() == 0);
+    }
+
+    // load dump file into another table
+    try (Table<byte[], byte[]> testTable2 = rdbStore.getTable("PrefixFifth")) {
+      testTable2.loadFromFile(dumpFile);
+
+      // check loaded keys
+      try (TableIterator<byte[],
+          ? extends Table.KeyValue<byte[], byte[]>> iter = testTable2.iterator(
+          samplePrefix)) {
+        int keyCount = 0;
+        while (iter.hasNext()) {
+          // check prefix
+          Assert.assertTrue(Arrays.equals(
+              Arrays.copyOf(iter.next().getKey(), PREFIX_LENGTH),
+              samplePrefix));
+          keyCount++;
+        }
+
+        // check block count
+        Assert.assertEquals(0, keyCount);
+      }
+    }
+  }
+
+  private List<String> generatePrefixes(int prefixCount) {
+    List<String> prefixes = new ArrayList<>();
+    for (int i = 0; i < prefixCount; i++) {
+      // use alphabetic chars so we get fixed length prefix when
+      // convert to byte[]
+      prefixes.add(RandomStringUtils.randomAlphabetic(PREFIX_LENGTH));
+    }
+    return prefixes;
+  }
+
+  private List<Map<String, String>> generateKVs(List<String> prefixes,
+      int keyCount) {
+    List<Map<String, String>> data = new ArrayList<>();
+    for (String prefix : prefixes) {
+      Map<String, String> kvs = new HashMap<>();
+      for (int i = 0; i < keyCount; i++) {
+        String key = prefix + i;
+        String val = RandomStringUtils.random(10);
+        kvs.put(key, val);
+      }
+      data.add(kvs);
+    }
+    return data;
+  }
+
+  private void populatePrefixedTable(Table<byte[], byte[]> table,
+      List<Map<String, String>> testData) throws IOException {
+    for (Map<String, String> segment : testData) {
+      for (Map.Entry<String, String> entry : segment.entrySet()) {
+        table.put(entry.getKey().getBytes(StandardCharsets.UTF_8),
+            entry.getValue().getBytes(StandardCharsets.UTF_8));
+      }
+    }
+  }
 }
diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto
index 3f6e999..c9d0c53 100644
--- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto
+++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto
@@ -147,6 +147,8 @@
   ERROR_IN_DB_SYNC = 42;
   CHUNK_FILE_INCONSISTENCY = 43;
   DELETE_ON_NON_EMPTY_CONTAINER = 44;
+  EXPORT_CONTAINER_METADATA_FAILED = 45;
+  IMPORT_CONTAINER_METADATA_FAILED = 46;
 }
 
 /**
diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
index f5d8267..e998b84 100644
--- a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
+++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
@@ -165,6 +165,7 @@
 message NodeReportProto {
   repeated StorageReportProto storageReport = 1;
   repeated MetadataStorageReportProto metadataStorageReport = 2;
+  repeated StorageReportProto dbStorageReport = 3;
 }
 
 message StorageReportProto {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMCertStore.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMCertStore.java
index d199c41..680bf43 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMCertStore.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMCertStore.java
@@ -265,7 +265,7 @@
     } else {
       List<? extends Table.KeyValue<BigInteger, CertInfo>> certs =
           scmMetadataStore.getRevokedCertsV2Table().getRangeKVs(
-          startSerialID, count);
+          startSerialID, count, null);
 
       for (Table.KeyValue<BigInteger, CertInfo> kv : certs) {
         try {
@@ -290,10 +290,10 @@
 
     if (role == SCM) {
       return scmMetadataStore.getValidSCMCertsTable().getRangeKVs(
-          startSerialID, count);
+          startSerialID, count, null);
     } else {
       return scmMetadataStore.getValidCertsTable().getRangeKVs(
-          startSerialID, count);
+          startSerialID, count, null);
     }
   }
 
diff --git a/hadoop-ozone/dist/src/main/compose/compatibility/test.sh b/hadoop-ozone/dist/src/main/compose/compatibility/test.sh
index 8dd51d0..b97f9d3 100755
--- a/hadoop-ozone/dist/src/main/compose/compatibility/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/compatibility/test.sh
@@ -32,6 +32,7 @@
 execute_robot_test om compatibility/om.robot
 execute_robot_test recon compatibility/recon.robot
 execute_robot_test scm compatibility/scm.robot
+execute_robot_test datanode compatibility/dn-one-rocksdb.robot
 
 stop_docker_env
 
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
index f46f067..3666730 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
@@ -70,7 +70,7 @@
 }
 
 with_new_version_finalized() {
-  _check_hdds_mlvs 3
+  _check_hdds_mlvs 4
   _check_om_mlvs 3
 
   validate old1
diff --git a/hadoop-ozone/dist/src/main/smoketest/compatibility/dn-one-rocksdb.robot b/hadoop-ozone/dist/src/main/smoketest/compatibility/dn-one-rocksdb.robot
new file mode 100644
index 0000000..1ef6d14
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/compatibility/dn-one-rocksdb.robot
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation       Test merge rocksdb in datanode compatibility
+Resource            ../commonlib.robot
+Test Timeout        5 minutes
+
+*** Variables ***
+
+
+*** Test Cases ***
+Create a container and check container schema version
+    ${output} =         Execute          ozone admin container create
+                        Should not contain  ${output}       Failed
+    ${output} =         Execute          ozone debug container list
+                        Should contain  ${output}    \"schemaVersion\" : \"2\"
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
index 4ae03ee..6b00312 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
@@ -25,12 +25,11 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
 import org.apache.hadoop.hdds.utils.MetadataKeyFilters.KeyPrefixFilter;
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
@@ -102,19 +101,20 @@
   public List<String> getPendingDeletionBlocks(Long containerID)
       throws IOException {
     List<String> pendingDeletionBlocks = Lists.newArrayList();
-    ReferenceCountedDB meta = getContainerMetadata(containerID);
-    KeyPrefixFilter filter =
-        new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX);
+    KeyValueContainerData cData = getContainerMetadata(containerID);
+    try (DBHandle db = BlockUtils.getDB(cData, conf)) {
+      KeyPrefixFilter filter = cData.getDeletingBlockKeyFilter();
 
-    List<? extends Table.KeyValue<String, BlockData>> kvs =
-        meta.getStore().getBlockDataTable()
-        .getRangeKVs(null, Integer.MAX_VALUE, filter);
+      List<? extends Table.KeyValue<String, BlockData>> kvs =
+          db.getStore().getBlockDataTable()
+              .getRangeKVs(cData.startKeyEmpty(), Integer.MAX_VALUE,
+                  cData.containerPrefix(), filter);
 
-    for (Table.KeyValue<String, BlockData> entry : kvs) {
-      pendingDeletionBlocks
-              .add(entry.getKey().replace(OzoneConsts.DELETING_KEY_PREFIX, ""));
+      for (Table.KeyValue<String, BlockData> entry : kvs) {
+        pendingDeletionBlocks
+            .add(entry.getKey().replace(cData.deletingBlockKeyPrefix(), ""));
+      }
     }
-    meta.close();
     return pendingDeletionBlocks;
   }
 
@@ -129,44 +129,46 @@
 
   public List<Long> getAllBlocks(Long containeID) throws IOException {
     List<Long> allBlocks = Lists.newArrayList();
-    ReferenceCountedDB meta = getContainerMetadata(containeID);
+    KeyValueContainerData cData = getContainerMetadata(containeID);
+    try (DBHandle db = BlockUtils.getDB(cData, conf)) {
 
-    List<? extends Table.KeyValue<String, BlockData>> kvs =
-          meta.getStore().getBlockDataTable()
-          .getRangeKVs(null, Integer.MAX_VALUE,
-          MetadataKeyFilters.getUnprefixedKeyFilter());
+      List<? extends Table.KeyValue<String, BlockData>> kvs =
+          db.getStore().getBlockDataTable()
+              .getRangeKVs(cData.startKeyEmpty(), Integer.MAX_VALUE,
+                  cData.containerPrefix(), cData.getUnprefixedKeyFilter());
 
-    for (Table.KeyValue<String, BlockData> entry : kvs) {
-      allBlocks.add(Long.valueOf(entry.getKey()));
+      for (Table.KeyValue<String, BlockData> entry : kvs) {
+        allBlocks.add(Long.valueOf(entry.getKey()));
+      }
     }
-    meta.close();
     return allBlocks;
   }
 
   public boolean verifyBlocksWithTxnTable(Map<Long, List<Long>> containerBlocks)
       throws IOException {
     for (Map.Entry<Long, List<Long>> entry : containerBlocks.entrySet()) {
-      ReferenceCountedDB meta = getContainerMetadata(entry.getKey());
-      DatanodeStore ds = meta.getStore();
-      DatanodeStoreSchemaTwoImpl dnStoreTwoImpl =
-          (DatanodeStoreSchemaTwoImpl) ds;
-      List<? extends Table.KeyValue<Long, DeletedBlocksTransaction>>
-          txnsInTxnTable = dnStoreTwoImpl.getDeleteTransactionTable()
-          .getRangeKVs(null, Integer.MAX_VALUE, null);
-      List<Long> conID = new ArrayList<>();
-      for (Table.KeyValue<Long, DeletedBlocksTransaction> txn :
-          txnsInTxnTable) {
-        conID.addAll(txn.getValue().getLocalIDList());
+      KeyValueContainerData cData = getContainerMetadata(entry.getKey());
+      try (DBHandle db = BlockUtils.getDB(cData, conf)) {
+        DatanodeStore ds = db.getStore();
+        DatanodeStoreSchemaTwoImpl dnStoreTwoImpl =
+            (DatanodeStoreSchemaTwoImpl) ds;
+        List<? extends Table.KeyValue<Long, DeletedBlocksTransaction>>
+            txnsInTxnTable = dnStoreTwoImpl.getDeleteTransactionTable()
+            .getRangeKVs(null, Integer.MAX_VALUE, null);
+        List<Long> conID = new ArrayList<>();
+        for (Table.KeyValue<Long, DeletedBlocksTransaction> txn :
+            txnsInTxnTable) {
+          conID.addAll(txn.getValue().getLocalIDList());
+        }
+        if (!conID.equals(containerBlocks.get(entry.getKey()))) {
+          return false;
+        }
       }
-      if (!conID.equals(containerBlocks.get(entry.getKey()))) {
-        return false;
-      }
-      meta.close();
     }
     return true;
   }
 
-  private ReferenceCountedDB getContainerMetadata(Long containerID)
+  private KeyValueContainerData getContainerMetadata(Long containerID)
       throws IOException {
     ContainerWithPipeline containerWithPipeline = cluster
         .getStorageContainerManager().getClientProtocolServer()
@@ -179,7 +181,7 @@
     KeyValueContainerData containerData =
         (KeyValueContainerData) containerServer.getContainerSet()
         .getContainer(containerID).getContainerData();
-    return BlockUtils.getDB(containerData, conf);
+    return containerData;
   }
 
   private OzoneContainer getContainerServerByDatanodeUuid(String dnUUID)
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClient.java
index 5ef50eb..e5c18d5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClient.java
@@ -54,7 +54,7 @@
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.container.TestHelper;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
@@ -267,10 +267,10 @@
         ((KeyValueContainer) cluster.getHddsDatanode(block1DNs.get(2))
             .getDatanodeStateMachine().getContainer().getContainerSet()
             .getContainer(containerId1)).getContainerData();
-    try (ReferenceCountedDB containerDb1 = BlockUtils.getDB(containerData1,
-        conf)) {
+    try (DBHandle containerDb1 = BlockUtils.getDB(containerData1, conf)) {
       BlockData blockData1 = containerDb1.getStore().getBlockDataTable().get(
-          Long.toString(locationList.get(0).getBlockID().getLocalID()));
+          containerData1.blockKey(locationList.get(0).getBlockID()
+              .getLocalID()));
       // The first Block could have 1 or 2 chunkSize of data
       int block1NumChunks = blockData1.getChunks().size();
       Assert.assertTrue(block1NumChunks >= 1);
@@ -286,10 +286,10 @@
         ((KeyValueContainer) cluster.getHddsDatanode(block2DNs.get(0))
             .getDatanodeStateMachine().getContainer().getContainerSet()
             .getContainer(containerId2)).getContainerData();
-    try (ReferenceCountedDB containerDb2 = BlockUtils.getDB(containerData2,
-        conf)) {
+    try (DBHandle containerDb2 = BlockUtils.getDB(containerData2, conf)) {
       BlockData blockData2 = containerDb2.getStore().getBlockDataTable().get(
-          Long.toString(locationList.get(1).getBlockID().getLocalID()));
+          containerData2.blockKey(locationList.get(1).getBlockID()
+              .getLocalID()));
       // The second Block should have 0.5 chunkSize of data
       Assert.assertEquals(block2ExpectedChunkCount,
           blockData2.getChunks().size());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
index e71a86d..b8fc543 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
@@ -84,7 +84,7 @@
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
@@ -1624,10 +1624,9 @@
         (KeyValueContainerData)(datanodeService.getDatanodeStateMachine()
             .getContainer().getContainerSet().getContainer(containerID)
             .getContainerData());
-    try (ReferenceCountedDB db = BlockUtils.getDB(containerData,
-            cluster.getConf());
+    try (DBHandle db = BlockUtils.getDB(containerData, cluster.getConf());
          BlockIterator<BlockData> keyValueBlockIterator =
-                db.getStore().getBlockIterator()) {
+                db.getStore().getBlockIterator(containerID)) {
       while (keyValueBlockIterator.hasNext()) {
         BlockData blockData = keyValueBlockIterator.nextBlock();
         if (blockData.getBlockID().getLocalID() == localID) {
@@ -1745,11 +1744,10 @@
       // Change first and second replica commit sequenceId
       if (index < 3) {
         long newBCSID = container.getBlockCommitSequenceId() - 1;
-        try (ReferenceCountedDB db = BlockUtils.getDB(
-            (KeyValueContainerData) container.getContainerData(),
-            cluster.getConf())) {
-          db.getStore().getMetadataTable().put(
-              OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID, newBCSID);
+        KeyValueContainerData cData =
+            (KeyValueContainerData) container.getContainerData();
+        try (DBHandle db = BlockUtils.getDB(cData, cluster.getConf())) {
+          db.getStore().getMetadataTable().put(cData.bcsIdKey(), newBCSID);
         }
         container.updateBlockCommitSequenceId(newBCSID);
         index++;
@@ -1885,10 +1883,9 @@
     // the container.
     KeyValueContainerData containerData =
         (KeyValueContainerData) container.getContainerData();
-    try (ReferenceCountedDB db = BlockUtils.getDB(containerData,
-            cluster.getConf());
+    try (DBHandle db = BlockUtils.getDB(containerData, cluster.getConf());
          BlockIterator<BlockData> keyValueBlockIterator =
-                 db.getStore().getBlockIterator()) {
+                 db.getStore().getBlockIterator(containerID)) {
       // Find the block corresponding to the key we put. We use the localID of
       // the BlockData to identify out key.
       BlockData blockData = null;
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestValidateBCSIDOnRestart.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestValidateBCSIDOnRestart.java
index 430f243..e70ffc5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestValidateBCSIDOnRestart.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestValidateBCSIDOnRestart.java
@@ -37,7 +37,6 @@
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
@@ -46,8 +45,8 @@
 import org.apache.hadoop.ozone.container.TestHelper;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
@@ -223,14 +222,13 @@
             .getContainerData();
     Assert.assertTrue(containerData instanceof KeyValueContainerData);
     keyValueContainerData = (KeyValueContainerData) containerData;
-    ReferenceCountedDB db = BlockUtils.
-            getDB(keyValueContainerData, conf);
+    try (DBHandle db = BlockUtils.getDB(keyValueContainerData, conf)) {
 
-    // modify the bcsid for the container in the ROCKS DB thereby inducing
-    // corruption
-    db.getStore().getMetadataTable()
-            .put(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID, 0L);
-    db.decrementReference();
+      // modify the bcsid for the container in the ROCKS DB thereby inducing
+      // corruption
+      db.getStore().getMetadataTable()
+          .put(keyValueContainerData.bcsIdKey(), 0L);
+    }
     // after the restart, there will be a mismatch in BCSID of what is recorded
     // in the and what is there in RockSDB and hence the container would be
     // marked unhealthy
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 98b524c..5bba351 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -38,7 +38,6 @@
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneTestUtils;
 import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -49,8 +48,8 @@
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.om.OzoneManager;
@@ -440,12 +439,11 @@
       ContainerSet dnContainerSet =
           datanode.getDatanodeStateMachine().getContainer().getContainerSet();
       OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
-        try (ReferenceCountedDB db = BlockUtils.getDB(
-            (KeyValueContainerData) dnContainerSet
-                .getContainer(blockID.getContainerID()).getContainerData(),
-            conf)) {
+        KeyValueContainerData cData = (KeyValueContainerData) dnContainerSet
+            .getContainer(blockID.getContainerID()).getContainerData();
+        try (DBHandle db = BlockUtils.getDB(cData, conf)) {
           Assert.assertNotNull(db.getStore().getBlockDataTable()
-              .get(Long.toString(blockID.getLocalID())));
+              .get(cData.blockKey(blockID.getLocalID())));
         }
       }, omKeyLocationInfoGroups);
     }
@@ -457,19 +455,18 @@
       ContainerSet dnContainerSet =
           datanode.getDatanodeStateMachine().getContainer().getContainerSet();
       OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
-        try (ReferenceCountedDB db = BlockUtils.getDB(
-            (KeyValueContainerData) dnContainerSet
-                .getContainer(blockID.getContainerID()).getContainerData(),
-            conf)) {
+        KeyValueContainerData cData = (KeyValueContainerData) dnContainerSet
+            .getContainer(blockID.getContainerID()).getContainerData();
+        try (DBHandle db = BlockUtils.getDB(cData, conf)) {
           Table<String, BlockData> blockDataTable =
               db.getStore().getBlockDataTable();
 
-          String blockIDString = Long.toString(blockID.getLocalID());
+          String blockKey = cData.blockKey(blockID.getLocalID());
 
-          BlockData blockData = blockDataTable.get(blockIDString);
+          BlockData blockData = blockDataTable.get(blockKey);
           Assert.assertNull(blockData);
 
-          String deletingKey = OzoneConsts.DELETING_KEY_PREFIX + blockIDString;
+          String deletingKey = cData.deletingBlockKey(blockID.getLocalID());
           Assert.assertNull(blockDataTable.get(deletingKey));
         }
         containerIdsWithDeletedBlocks.add(blockID.getContainerID());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
index 3a39fe1..c1d4fa3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
@@ -34,6 +34,7 @@
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
@@ -41,7 +42,6 @@
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 import org.apache.ozone.test.GenericTestUtils;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.Assert;
 import org.junit.jupiter.api.BeforeAll;
@@ -250,7 +250,7 @@
     List<DatanodeDetails> datanodes = pipeline.getNodes();
     Assert.assertEquals(3, datanodes.size());
 
-    List<ReferenceCountedDB> metadataStores = new ArrayList<>(datanodes.size());
+    List<DBHandle> metadataStores = new ArrayList<>(datanodes.size());
     for (DatanodeDetails details : datanodes) {
       Assert.assertFalse(isContainerClosed(cluster, containerID, details));
       //send the order to close the container
@@ -264,7 +264,7 @@
       Container dnContainer = cluster.getHddsDatanodes().get(index)
           .getDatanodeStateMachine().getContainer().getContainerSet()
           .getContainer(containerID);
-      try (ReferenceCountedDB store = BlockUtils.getDB(
+      try (DBHandle store = BlockUtils.getDB(
           (KeyValueContainerData) dnContainer.getContainerData(), conf)) {
         metadataStores.add(store);
       }
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBDefinitionFactory.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBDefinitionFactory.java
index 271a03c..369da54 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBDefinitionFactory.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBDefinitionFactory.java
@@ -22,9 +22,11 @@
 import java.util.Arrays;
 import java.util.HashMap;
 
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
 import org.apache.hadoop.hdds.utils.db.DBDefinition;
 import org.apache.hadoop.ozone.container.metadata.DatanodeSchemaOneDBDefinition;
+import org.apache.hadoop.ozone.container.metadata.DatanodeSchemaThreeDBDefinition;
 import org.apache.hadoop.ozone.container.metadata.DatanodeSchemaTwoDBDefinition;
 import org.apache.hadoop.ozone.om.codec.OMDBDefinition;
 import org.apache.hadoop.ozone.recon.scm.ReconSCMDBDefinition;
@@ -63,7 +65,8 @@
     return getReconDBDefinition(dbName);
   }
 
-  public static DBDefinition getDefinition(Path dbPath) {
+  public static DBDefinition getDefinition(Path dbPath,
+      ConfigurationSource config) {
     Preconditions.checkNotNull(dbPath,
         "Path is required to identify the used db scheme");
     final Path fileName = dbPath.getFileName();
@@ -72,14 +75,17 @@
           "Path is required to identify the used db scheme");
     }
     String dbName = fileName.toString();
-    if (dbName.endsWith("-container.db")) {
+    if (dbName.endsWith("container.db")) {
       switch (dnDBSchemaVersion) {
       case "V1":
         return new DatanodeSchemaOneDBDefinition(
-            dbPath.toAbsolutePath().toString());
+            dbPath.toAbsolutePath().toString(), config);
+      case "V3":
+        return new DatanodeSchemaThreeDBDefinition(
+            dbPath.toAbsolutePath().toString(), config);
       default:
         return new DatanodeSchemaTwoDBDefinition(
-            dbPath.toAbsolutePath().toString());
+            dbPath.toAbsolutePath().toString(), config);
       }
     }
     return getDefinition(dbName);
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBScanner.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBScanner.java
index 275908e..94cbb85 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBScanner.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/DBScanner.java
@@ -32,6 +32,7 @@
 import java.util.concurrent.Callable;
 
 import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
 import org.apache.hadoop.hdds.utils.db.DBDefinition;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -216,7 +217,7 @@
     dbPath = removeTrailingSlashIfNeeded(dbPath);
     DBDefinitionFactory.setDnDBSchemaVersion(dnDBSchemaVersion);
     this.constructColumnFamilyMap(DBDefinitionFactory.
-            getDefinition(Paths.get(dbPath)));
+            getDefinition(Paths.get(dbPath), new OzoneConfiguration()));
     if (this.columnFamilyMap != null) {
       if (!this.columnFamilyMap.containsKey(tableName)) {
         System.out.print("Table with name:" + tableName + " does not exist");
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/PrefixParser.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/PrefixParser.java
index bf3ccbe..6e5ce01 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/PrefixParser.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/PrefixParser.java
@@ -226,7 +226,7 @@
 
     List<? extends KeyValue
         <String, ? extends WithParentObjectId>> infoList =
-        table.getRangeKVs(null, 1000, filter);
+        table.getRangeKVs(null, 1000, null, filter);
 
     for (KeyValue<String, ? extends WithParentObjectId> info :infoList) {
       Path key = Paths.get(info.getKey());
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/container/ContainerCommands.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/container/ContainerCommands.java
index 1153c4d..c5b21d6 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/container/ContainerCommands.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/container/ContainerCommands.java
@@ -35,7 +35,6 @@
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.interfaces.Handler;
-import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
 import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
@@ -191,7 +190,7 @@
           "Version file " + versionFile + " is missing");
     }
 
-    return HddsVolumeUtil
+    return StorageVolumeUtil
         .getProperty(props, OzoneConsts.DATANODE_UUID, versionFile);
   }
 
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/containergenerator/GeneratorDatanode.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/containergenerator/GeneratorDatanode.java
index b2fa7d4..f9ed369 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/containergenerator/GeneratorDatanode.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/containergenerator/GeneratorDatanode.java
@@ -52,7 +52,7 @@
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
 import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
 import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext.WriteChunkStage;
-import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
@@ -161,9 +161,9 @@
           "Version file " + versionFile + " is missing");
     }
 
-    String clusterId =
-        HddsVolumeUtil.getProperty(props, OzoneConsts.CLUSTER_ID, versionFile);
-    datanodeId = HddsVolumeUtil
+    String clusterId = StorageVolumeUtil.getProperty(props,
+        OzoneConsts.CLUSTER_ID, versionFile);
+    datanodeId = StorageVolumeUtil
         .getProperty(props, OzoneConsts.DATANODE_UUID, versionFile);
 
     volumeSet = new MutableVolumeSet(datanodeId, clusterId, config, null,
diff --git a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/debug/TestDBDefinitionFactory.java b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/debug/TestDBDefinitionFactory.java
index 4d018d1..6ba9faa 100644
--- a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/debug/TestDBDefinitionFactory.java
+++ b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/debug/TestDBDefinitionFactory.java
@@ -20,9 +20,11 @@
 
 import java.nio.file.Paths;
 
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
 import org.apache.hadoop.hdds.utils.db.DBDefinition;
 import org.apache.hadoop.ozone.container.metadata.DatanodeSchemaOneDBDefinition;
+import org.apache.hadoop.ozone.container.metadata.DatanodeSchemaThreeDBDefinition;
 import org.apache.hadoop.ozone.container.metadata.DatanodeSchemaTwoDBDefinition;
 import org.apache.hadoop.ozone.om.codec.OMDBDefinition;
 import org.apache.hadoop.ozone.recon.scm.ReconSCMDBDefinition;
@@ -61,11 +63,18 @@
     assertTrue(definition instanceof ReconDBDefinition);
     DBDefinitionFactory.setDnDBSchemaVersion("V2");
     definition =
-        DBDefinitionFactory.getDefinition(Paths.get("/tmp/test-container.db"));
+        DBDefinitionFactory.getDefinition(Paths.get("/tmp/test-container.db"),
+            new OzoneConfiguration());
     assertTrue(definition instanceof DatanodeSchemaTwoDBDefinition);
     DBDefinitionFactory.setDnDBSchemaVersion("V1");
     definition =
-        DBDefinitionFactory.getDefinition(Paths.get("/tmp/test-container.db"));
+        DBDefinitionFactory.getDefinition(Paths.get("/tmp/test-container.db"),
+            new OzoneConfiguration());
     assertTrue(definition instanceof DatanodeSchemaOneDBDefinition);
+    DBDefinitionFactory.setDnDBSchemaVersion("V3");
+    definition =
+        DBDefinitionFactory.getDefinition(Paths.get("/tmp/test-container.db"),
+            new OzoneConfiguration());
+    assertTrue(definition instanceof DatanodeSchemaThreeDBDefinition);
   }
 }
\ No newline at end of file