You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ms...@apache.org on 2022/03/02 10:36:12 UTC

[ozone] branch HDDS-3630 updated: HDDS-5923. Add DBHandle abstraction over ReferenceCountedDB. (#2794)

This is an automated email from the ASF dual-hosted git repository.

msingh pushed a commit to branch HDDS-3630
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/HDDS-3630 by this push:
     new 1146247  HDDS-5923. Add DBHandle abstraction over ReferenceCountedDB. (#2794)
1146247 is described below

commit 1146247779a636b9071a53a16731ad61a57e150f
Author: Gui Hecheng <ma...@tencent.com>
AuthorDate: Wed Mar 2 18:30:39 2022 +0800

    HDDS-5923. Add DBHandle abstraction over ReferenceCountedDB. (#2794)
---
 .../container/common/interfaces/DBHandle.java      | 48 +++++++++++++++
 .../commandhandler/DeleteBlocksCommandHandler.java | 10 ++--
 .../hadoop/ozone/container/common/utils/RawDB.java | 42 +++++++++++++
 .../container/common/utils/ReferenceCountedDB.java | 25 +++-----
 .../container/keyvalue/KeyValueContainer.java      |  6 +-
 .../container/keyvalue/KeyValueContainerCheck.java |  5 +-
 .../container/keyvalue/KeyValueContainerData.java  |  5 +-
 .../container/keyvalue/helpers/BlockUtils.java     | 13 +++--
 .../keyvalue/helpers/KeyValueContainerUtil.java    |  8 +--
 .../container/keyvalue/impl/BlockManagerImpl.java  | 12 ++--
 .../background/BlockDeletingService.java           |  8 +--
 .../container/common/TestBlockDeletingService.java | 16 ++---
 .../TestSchemaOneBackwardsCompatibility.java       | 34 +++++------
 .../common/impl/TestContainerPersistence.java      |  4 +-
 .../keyvalue/TestKeyValueBlockIterator.java        |  7 +--
 .../container/keyvalue/TestKeyValueContainer.java  | 13 ++---
 .../keyvalue/TestKeyValueContainerCheck.java       |  5 +-
 .../TestKeyValueContainerIntegrityChecks.java      |  4 +-
 .../TestKeyValueContainerMetadataInspector.java    |  6 +-
 .../container/ozoneimpl/TestContainerReader.java   |  6 +-
 .../container/ozoneimpl/TestOzoneContainer.java    |  5 +-
 .../ozone/TestStorageContainerManagerHelper.java   | 68 +++++++++++-----------
 .../client/rpc/TestFailureHandlingByClient.java    |  8 +--
 .../client/rpc/TestOzoneRpcClientAbstract.java     | 10 ++--
 .../client/rpc/TestValidateBCSIDOnRestart.java     | 17 +++---
 .../commandhandler/TestBlockDeletion.java          |  6 +-
 .../TestCloseContainerByPipeline.java              |  6 +-
 27 files changed, 230 insertions(+), 167 deletions(-)

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/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
index f713db8..314ca7d 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
@@ -37,6 +37,7 @@ 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;
@@ -53,7 +54,6 @@ import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 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;
 
@@ -332,8 +332,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
     if (!isTxnIdValid(containerId, containerData, delTX)) {
       return;
     }
-    try (ReferenceCountedDB containerDB = BlockUtils
-        .getDB(containerData, conf)) {
+    try (DBHandle containerDB = BlockUtils.getDB(containerData, conf)) {
       DatanodeStore ds = containerDB.getStore();
       DatanodeStoreSchemaTwoImpl dnStoreTwoImpl =
           (DatanodeStoreSchemaTwoImpl) ds;
@@ -358,8 +357,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
       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 =
@@ -412,7 +410,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
 
   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.
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 @@ package org.apache.hadoop.ozone.container.common.utils;
 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 @@ import java.util.concurrent.atomic.AtomicInteger;
  * 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 class ReferenceCountedDB implements Closeable {
   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 @@ public class ReferenceCountedDB implements Closeable {
     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 class ReferenceCountedDB implements Closeable {
     }
   }
 
-  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/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index fe087b0..2eec14f 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.helpers.ContainerUtils;
 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;
@@ -408,7 +408,7 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
 
   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 +422,7 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
 
   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(),
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 b5c68d3..1c1e558 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,10 @@ import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 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.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;
@@ -230,8 +230,7 @@ public class KeyValueContainerCheck {
 
     ContainerLayoutVersion layout = onDiskContainerData.getLayoutVersion();
 
-    try (ReferenceCountedDB db =
-            BlockUtils.getDB(onDiskContainerData, checkConfig);
+    try (DBHandle db = BlockUtils.getDB(onDiskContainerData, checkConfig);
         BlockIterator<BlockData> kvIter = db.getStore().getBlockIterator(
             onDiskContainerData.getContainerID())) {
 
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 ccc252a..594f3b2 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
@@ -32,7 +32,7 @@ import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.hdds.utils.db.Table;
 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;
 
 
@@ -286,9 +286,10 @@ public class KeyValueContainerData extends ContainerData {
    * @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();
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 b6a5b6a..4dccba6 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
@@ -26,6 +26,7 @@ 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.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
@@ -108,9 +109,8 @@ public final class BlockUtils {
    * @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);
@@ -153,15 +153,16 @@ public final class BlockUtils {
   /**
    * 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.
    */
-  public static void addDB(ReferenceCountedDB db, String containerDBPath,
+  public static void addDB(DatanodeStore store, String containerDBPath,
       ConfigurationSource conf) {
     ContainerCache cache = ContainerCache.getInstance(conf);
     Preconditions.checkNotNull(cache);
-    cache.addDB(containerDBPath, db);
+    cache.addDB(containerDBPath,
+        new ReferenceCountedDB(store, containerDBPath));
   }
 
   /**
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 0a30c36..1958a3e 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.ChunkInfo;
 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;
@@ -114,10 +114,8 @@ public final class KeyValueContainerUtil {
               "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);
   }
 
   /**
@@ -187,7 +185,7 @@ public final class KeyValueContainerUtil {
     }
 
     boolean isBlockMetadataSet = false;
-    ReferenceCountedDB cachedDB = null;
+    DBHandle cachedDB = null;
     DatanodeStore store = null;
     try {
       try {
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 ee13ffc..d8407e6 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
@@ -33,8 +33,8 @@ 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.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.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
@@ -119,7 +119,7 @@ public class BlockManagerImpl implements BlockManager {
         "cannot be negative");
     // 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.
+    try (DBHandle db = BlockUtils.
         getDB(container.getContainerData(), config)) {
       // This is a post condition that acts as a hint to the user.
       // Should never fail.
@@ -251,7 +251,7 @@ public class BlockManagerImpl implements BlockManager {
               + 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);
@@ -279,7 +279,7 @@ public class BlockManagerImpl implements BlockManager {
       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);
@@ -326,7 +326,7 @@ public class BlockManagerImpl implements BlockManager {
       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<>();
         List<? extends Table.KeyValue<String, BlockData>> range =
             db.getStore().getBlockDataTable()
@@ -351,7 +351,7 @@ public class BlockManagerImpl implements BlockManager {
     BlockUtils.shutdownCache(ContainerCache.getInstance(config));
   }
 
-  private BlockData getBlockByID(ReferenceCountedDB db, BlockID blockID)
+  private BlockData getBlockByID(DBHandle db, BlockID blockID)
       throws IOException {
     String blockKey = Long.toString(blockID.getLocalID());
 
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..8206c29 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
@@ -48,9 +48,9 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 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;
@@ -278,7 +278,7 @@ public class BlockDeletingService extends BackgroundService {
       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)) {
@@ -304,7 +304,7 @@ public class BlockDeletingService extends BackgroundService {
     }
 
     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)) {
@@ -394,7 +394,7 @@ public class BlockDeletingService extends BackgroundService {
     }
 
     public ContainerBackgroundTaskResult deleteViaSchema2(
-        ReferenceCountedDB meta, Container container, File dataDir,
+        DBHandle meta, Container container, File dataDir,
         long startTime) throws IOException {
       ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult();
       if (!checkDataDir(dataDir)) {
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..ce644ce 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
@@ -56,8 +56,8 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 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;
@@ -236,7 +236,7 @@ public class TestBlockDeletingService {
       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 =
@@ -272,7 +272,7 @@ public class TestBlockDeletingService {
           container, blockID);
       kd.setChunks(chunks);
       String bID = null;
-      try (ReferenceCountedDB metadata = BlockUtils.getDB(data, conf)) {
+      try (DBHandle metadata = BlockUtils.getDB(data, conf)) {
         bID = blockID.getLocalID() + "";
         metadata.getStore().getBlockDataTable().put(bID, kd);
       } catch (IOException exception) {
@@ -294,7 +294,7 @@ public class TestBlockDeletingService {
 
   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)
@@ -344,7 +344,7 @@ public class TestBlockDeletingService {
       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()
@@ -375,7 +375,7 @@ public class TestBlockDeletingService {
    * 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()
@@ -428,7 +428,7 @@ public class TestBlockDeletingService {
     KeyValueContainerData data = (KeyValueContainerData) containerData.get(0);
     Assert.assertEquals(1, containerData.size());
 
-    try (ReferenceCountedDB meta = BlockUtils.getDB(
+    try (DBHandle meta = BlockUtils.getDB(
         (KeyValueContainerData) containerData.get(0), conf)) {
       Map<Long, Container<?>> containerMap = containerSet.getContainerMapCopy();
       // NOTE: this test assumes that all the container is KetValueContainer and
@@ -560,7 +560,7 @@ public class TestBlockDeletingService {
     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/TestSchemaOneBackwardsCompatibility.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
index cea7fa9..ec46bd5 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
@@ -33,7 +33,7 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 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;
@@ -130,8 +130,7 @@ public class TestSchemaOneBackwardsCompatibility {
    */
   @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,8 +158,7 @@ public class TestSchemaOneBackwardsCompatibility {
    */
   @Test
   public void testBlockIteration() throws IOException {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    try (DBHandle refCountedDB = BlockUtils.getDB(newKvData(), conf)) {
       assertEquals(TestDB.NUM_DELETED_BLOCKS, countDeletedBlocks(refCountedDB));
 
       assertEquals(TestDB.NUM_PENDING_DELETION_BLOCKS,
@@ -217,7 +215,7 @@ public class TestSchemaOneBackwardsCompatibility {
   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)) {
+    try (DBHandle db = BlockUtils.getDB(newKvData(), conf)) {
       Table<String, Long> metadataTable = db.getStore().getMetadataTable();
 
       metadataTable.delete(OzoneConsts.BLOCK_COUNT);
@@ -280,8 +278,7 @@ public class TestSchemaOneBackwardsCompatibility {
     final long expectedRegularBlocks =
             TestDB.KEY_COUNT - numBlocksToDelete;
 
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    try (DBHandle refCountedDB = BlockUtils.getDB(newKvData(), conf)) {
       // Test results via block iteration.
 
       assertEquals(expectedDeletingBlocks,
@@ -323,8 +320,7 @@ public class TestSchemaOneBackwardsCompatibility {
         new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet,
             metrics, c -> {
         });
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    try (DBHandle refCountedDB = BlockUtils.getDB(newKvData(), conf)) {
       // Read blocks that were already deleted before the upgrade.
       List<? extends Table.KeyValue<String, ChunkInfoList>> deletedBlocks =
               refCountedDB.getStore()
@@ -374,8 +370,7 @@ public class TestSchemaOneBackwardsCompatibility {
 
   @Test
   public void testReadBlockData() throws Exception {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    try (DBHandle refCountedDB = BlockUtils.getDB(newKvData(), conf)) {
       Table<String, BlockData> blockDataTable =
           refCountedDB.getStore().getBlockDataTable();
 
@@ -417,8 +412,7 @@ public class TestSchemaOneBackwardsCompatibility {
 
   @Test
   public void testReadDeletingBlockData() throws Exception {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    try (DBHandle refCountedDB = BlockUtils.getDB(newKvData(), conf)) {
       Table<String, BlockData> blockDataTable =
           refCountedDB.getStore().getBlockDataTable();
 
@@ -470,8 +464,7 @@ public class TestSchemaOneBackwardsCompatibility {
 
   @Test
   public void testReadMetadata() throws Exception {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    try (DBHandle refCountedDB = BlockUtils.getDB(newKvData(), conf)) {
       Table<String, Long> metadataTable =
           refCountedDB.getStore().getMetadataTable();
 
@@ -487,8 +480,7 @@ public class TestSchemaOneBackwardsCompatibility {
 
   @Test
   public void testReadDeletedBlocks() throws Exception {
-    try (ReferenceCountedDB refCountedDB =
-        BlockUtils.getDB(newKvData(), conf)) {
+    try (DBHandle refCountedDB = BlockUtils.getDB(newKvData(), conf)) {
       Table<String, ChunkInfoList> deletedBlocksTable =
           refCountedDB.getStore().getDeletedBlocksTable();
 
@@ -593,21 +585,21 @@ public class TestSchemaOneBackwardsCompatibility {
             kvData.getNumPendingDeletionBlocks());
   }
 
-  private int countDeletedBlocks(ReferenceCountedDB refCountedDB)
+  private int countDeletedBlocks(DBHandle refCountedDB)
           throws IOException {
     return refCountedDB.getStore().getDeletedBlocksTable()
             .getRangeKVs(null, 100,
                     MetadataKeyFilters.getUnprefixedKeyFilter()).size();
   }
 
-  private int countDeletingBlocks(ReferenceCountedDB refCountedDB)
+  private int countDeletingBlocks(DBHandle refCountedDB)
           throws IOException {
     return refCountedDB.getStore().getBlockDataTable()
             .getRangeKVs(null, 100,
                     MetadataKeyFilters.getDeletingKeyFilter()).size();
   }
 
-  private int countUnprefixedBlocks(ReferenceCountedDB refCountedDB)
+  private int countUnprefixedBlocks(DBHandle refCountedDB)
           throws IOException {
     return refCountedDB.getStore().getBlockDataTable()
             .getRangeKVs(null, 100,
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..954e5be 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,9 +48,9 @@ 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.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;
@@ -220,7 +220,7 @@ public class TestContainerPersistence {
     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);
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 1e13446..655f400 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
@@ -42,7 +42,7 @@ 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.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.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
@@ -75,7 +75,7 @@ public class TestKeyValueBlockIterator {
   private MutableVolumeSet volumeSet;
   private OzoneConfiguration conf;
   private File testRoot;
-  private ReferenceCountedDB db;
+  private DBHandle db;
   private final ContainerLayoutVersion layout;
 
   public TestKeyValueBlockIterator(ContainerLayoutVersion layout) {
@@ -372,8 +372,7 @@ public class TestKeyValueBlockIterator {
             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);
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 4f9e45c..ea009b7 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
@@ -33,6 +33,7 @@ import org.apache.hadoop.ozone.OzoneConsts;
 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;
@@ -46,7 +47,6 @@ import org.apache.hadoop.ozone.container.metadata.AbstractDatanodeStore;
 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;
@@ -321,7 +321,7 @@ public class TestKeyValueContainer {
    * Add some keys to the container.
    */
   private void populate(long numberOfKeysToWrite) throws IOException {
-    try (ReferenceCountedDB metadataStore =
+    try (DBHandle metadataStore =
         BlockUtils.getDB(keyValueContainer.getContainerData(), CONF)) {
       Table<String, BlockData> blockDataTable =
               metadataStore.getStore().getBlockDataTable();
@@ -505,14 +505,13 @@ public class TestKeyValueContainer {
 
   @Test
   public void testContainerRocksDB()
-      throws StorageContainerException, RocksDBException {
+      throws IOException, RocksDBException {
     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
@@ -565,7 +564,7 @@ public class TestKeyValueContainer {
     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);
@@ -585,7 +584,7 @@ public class TestKeyValueContainer {
     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 2fb9c8b..a37684f 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.hdfs.util.DataTransferThrottler;
 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;
@@ -112,8 +112,7 @@ public class TestKeyValueContainerCheck
     File dbFile = KeyValueContainerLocationUtil
         .getContainerDBFile(metaDir, containerID);
     containerData.setDbFile(dbFile);
-    try (ReferenceCountedDB ignored =
-            BlockUtils.getDB(containerData, conf);
+    try (DBHandle ignored = BlockUtils.getDB(containerData, conf);
         BlockIterator<BlockData> kvIter =
                 ignored.getStore().getBlockIterator(containerID)) {
       BlockData block = kvIter.nextBlock();
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..a4ee503 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
@@ -29,6 +29,7 @@ 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 +37,6 @@ import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingP
 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;
@@ -140,7 +140,7 @@ public class TestKeyValueContainerIntegrityChecks {
     KeyValueContainer container = new KeyValueContainer(containerData, conf);
     container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
         UUID.randomUUID().toString());
-    try (ReferenceCountedDB metadataStore = BlockUtils.getDB(containerData,
+    try (DBHandle metadataStore = BlockUtils.getDB(containerData,
         conf)) {
       assertNotNull(containerData.getChunksPath());
       File chunksPath = new File(containerData.getChunksPath());
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..b7334ac 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 com.google.gson.JsonPrimitive;
 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;
@@ -290,7 +290,7 @@ public class TestKeyValueContainerMetadataInspector
 
   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);
@@ -300,7 +300,7 @@ public class TestKeyValueContainerMetadataInspector
 
   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);
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..ee1ca75 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,8 +31,8 @@ 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.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.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
@@ -124,7 +124,7 @@ public class TestContainerReader {
 
   private void markBlocksForDelete(KeyValueContainer keyValueContainer,
       boolean setMetaData, List<Long> blockNames, int count) throws Exception {
-    try (ReferenceCountedDB metadataStore = BlockUtils.getDB(keyValueContainer
+    try (DBHandle metadataStore = BlockUtils.getDB(keyValueContainer
         .getContainerData(), conf)) {
 
       for (int i = 0; i < count; i++) {
@@ -154,7 +154,7 @@ public class TestContainerReader {
     long containerId = keyValueContainer.getContainerData().getContainerID();
 
     List<Long> blkNames = new ArrayList<>();
-    try (ReferenceCountedDB metadataStore = BlockUtils.getDB(keyValueContainer
+    try (DBHandle metadataStore = BlockUtils.getDB(keyValueContainer
         .getContainerData(), conf)) {
 
       for (int i = 0; i < blockCount; i++) {
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..c9d79e8 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
@@ -35,9 +35,9 @@ 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;
@@ -277,8 +277,7 @@ public class TestOzoneContainer {
 
     long freeBytes = container.getContainerData().getMaxSize();
     long containerId = container.getContainerData().getContainerID();
-    ReferenceCountedDB db = BlockUtils.getDB(container
-        .getContainerData(), conf);
+    DBHandle db = BlockUtils.getDB(container.getContainerData(), conf);
 
     Table<String, Long> metadataTable = db.getStore().getMetadataTable();
     Table<String, BlockData> blockDataTable = db.getStore().getBlockDataTable();
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..cdecca3 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
@@ -30,7 +30,7 @@ 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 +102,19 @@ public class TestStorageContainerManagerHelper {
   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);
+    try (DBHandle meta = getContainerMetadata(containerID)) {
+      KeyPrefixFilter filter =
+          new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX);
 
-    List<? extends Table.KeyValue<String, BlockData>> kvs =
-        meta.getStore().getBlockDataTable()
-        .getRangeKVs(null, Integer.MAX_VALUE, filter);
+      List<? extends Table.KeyValue<String, BlockData>> kvs =
+          meta.getStore().getBlockDataTable()
+              .getRangeKVs(null, Integer.MAX_VALUE, 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(OzoneConsts.DELETING_KEY_PREFIX, ""));
+      }
     }
-    meta.close();
     return pendingDeletionBlocks;
   }
 
@@ -129,44 +129,44 @@ public class TestStorageContainerManagerHelper {
 
   public List<Long> getAllBlocks(Long containeID) throws IOException {
     List<Long> allBlocks = Lists.newArrayList();
-    ReferenceCountedDB meta = getContainerMetadata(containeID);
+    try (DBHandle meta = getContainerMetadata(containeID)) {
 
-    List<? extends Table.KeyValue<String, BlockData>> kvs =
+      List<? extends Table.KeyValue<String, BlockData>> kvs =
           meta.getStore().getBlockDataTable()
-          .getRangeKVs(null, Integer.MAX_VALUE,
-          MetadataKeyFilters.getUnprefixedKeyFilter());
+              .getRangeKVs(null, Integer.MAX_VALUE,
+                  MetadataKeyFilters.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());
-      }
-      if (!conID.equals(containerBlocks.get(entry.getKey()))) {
-        return false;
+      try (DBHandle 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());
+        }
+        if (!conID.equals(containerBlocks.get(entry.getKey()))) {
+          return false;
+        }
       }
-      meta.close();
     }
     return true;
   }
 
-  private ReferenceCountedDB getContainerMetadata(Long containerID)
+  private DBHandle getContainerMetadata(Long containerID)
       throws IOException {
     ContainerWithPipeline containerWithPipeline = cluster
         .getStorageContainerManager().getClientProtocolServer()
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 41a0892..a9f4621 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.client.io.OzoneOutputStream;
 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;
@@ -273,8 +273,7 @@ public class TestFailureHandlingByClient {
         ((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()));
       // The first Block could have 1 or 2 chunkSize of data
@@ -292,8 +291,7 @@ public class TestFailureHandlingByClient {
         ((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()));
       // The second Block should have 0.5 chunkSize of data
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 e638c13..fd7355b 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
@@ -79,7 +79,7 @@ import org.apache.hadoop.ozone.common.OzoneChecksumException;
 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;
@@ -1478,8 +1478,7 @@ public abstract class TestOzoneRpcClientAbstract {
         (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(containerID)) {
       while (keyValueBlockIterator.hasNext()) {
@@ -1598,7 +1597,7 @@ public abstract class TestOzoneRpcClientAbstract {
       // Change first and second replica commit sequenceId
       if (index < 3) {
         long newBCSID = container.getBlockCommitSequenceId() - 1;
-        try (ReferenceCountedDB db = BlockUtils.getDB(
+        try (DBHandle db = BlockUtils.getDB(
             (KeyValueContainerData) container.getContainerData(),
             cluster.getConf())) {
           db.getStore().getMetadataTable().put(
@@ -1738,8 +1737,7 @@ public abstract class TestOzoneRpcClientAbstract {
     // 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(containerID)) {
       // Find the block corresponding to the key we put. We use the localID of
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..cdaf4c1 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
@@ -46,8 +46,8 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 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 +223,13 @@ public class TestValidateBCSIDOnRestart {
             .getContainerData();
     Assert.assertTrue(containerData instanceof KeyValueContainerData);
     keyValueContainerData = (KeyValueContainerData) containerData;
-    ReferenceCountedDB 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();
+    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);
+    }
     // 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 8978350..73f5fc0 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
@@ -49,6 +49,7 @@ import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 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.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
@@ -59,7 +60,6 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
 import org.apache.ozone.test.GenericTestUtils;
 import org.apache.ozone.test.GenericTestUtils.LogCapturer;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -439,7 +439,7 @@ public class TestBlockDeletion {
       ContainerSet dnContainerSet =
           datanode.getDatanodeStateMachine().getContainer().getContainerSet();
       OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
-        try (ReferenceCountedDB db = BlockUtils.getDB(
+        try (DBHandle db = BlockUtils.getDB(
             (KeyValueContainerData) dnContainerSet
                 .getContainer(blockID.getContainerID()).getContainerData(),
             conf)) {
@@ -456,7 +456,7 @@ public class TestBlockDeletion {
       ContainerSet dnContainerSet =
           datanode.getDatanodeStateMachine().getContainer().getContainerSet();
       OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
-        try (ReferenceCountedDB db = BlockUtils.getDB(
+        try (DBHandle db = BlockUtils.getDB(
             (KeyValueContainerData) dnContainerSet
                 .getContainer(blockID.getContainerID()).getContainerData(),
             conf)) {
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 d629f2f..afe0cbd 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.OzoneClientFactory;
 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.om.helpers.OmKeyLocationInfo;
 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.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -249,7 +249,7 @@ public class TestCloseContainerByPipeline {
     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
@@ -263,7 +263,7 @@ public class TestCloseContainerByPipeline {
       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);
       }

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