You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ha...@apache.org on 2020/10/14 23:54:19 UTC

[hadoop-ozone] branch master updated: HDDS-4327. Potential resource leakage using BatchOperation. (#1493)

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

hanishakoneru pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hadoop-ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 342bf6d  HDDS-4327. Potential resource leakage using BatchOperation. (#1493)
342bf6d is described below

commit 342bf6de8a18485a4c31923eb014cfd63c194537
Author: Bharat Viswanadham <bh...@apache.org>
AuthorDate: Wed Oct 14 16:54:03 2020 -0700

    HDDS-4327. Potential resource leakage using BatchOperation. (#1493)
---
 .../commandhandler/DeleteBlocksCommandHandler.java | 65 ++++++++++----------
 .../container/keyvalue/impl/BlockManagerImpl.java  | 70 +++++++++++-----------
 .../background/BlockDeletingService.java           | 45 +++++++-------
 .../hadoop/hdds/scm/block/DeletedBlockLogImpl.java | 19 +++---
 .../hdds/scm/container/SCMContainerManager.java    | 28 ++++-----
 .../org/apache/hadoop/ozone/om/OzoneManager.java   | 15 ++---
 .../key/TestOMKeyPurgeRequestAndResponse.java      | 15 ++---
 .../bucket/TestOMBucketCreateResponse.java         |  8 +++
 .../bucket/TestOMBucketDeleteResponse.java         |  8 +++
 .../bucket/TestOMBucketSetPropertyResponse.java    |  9 +++
 .../file/TestOMDirectoryCreateResponse.java        |  8 +++
 .../ozone/om/response/key/TestOMKeyResponse.java   |  3 +
 .../s3/multipart/TestS3MultipartResponse.java      |  9 +++
 .../security/TestOMDelegationTokenResponse.java    |  8 +++
 .../volume/TestOMVolumeCreateResponse.java         |  8 +++
 .../volume/TestOMVolumeDeleteResponse.java         |  8 +++
 .../volume/TestOMVolumeSetOwnerResponse.java       |  9 +++
 .../volume/TestOMVolumeSetQuotaResponse.java       |  9 +++
 18 files changed, 220 insertions(+), 124 deletions(-)

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 ebccf4d..91ab4c9 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
@@ -215,8 +215,6 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
 
       for (Long blkLong : delTX.getLocalIDList()) {
         String blk = blkLong.toString();
-        BatchOperation batch = containerDB.getStore()
-                .getBatchHandler().initBatchOperation();
         BlockData blkInfo = blockDataTable.get(blk);
         if (blkInfo != null) {
           String deletingKey = OzoneConsts.DELETING_KEY_PREFIX + blk;
@@ -230,13 +228,15 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
             }
             continue;
           }
-          // Found the block in container db,
-          // use an atomic update to change its state to deleting.
-          blockDataTable.putWithBatch(batch, deletingKey, blkInfo);
-          blockDataTable.deleteWithBatch(batch, blk);
-          try {
+
+          try(BatchOperation batch = containerDB.getStore()
+              .getBatchHandler().initBatchOperation()) {
+            // Found the block in container db,
+            // use an atomic update to change its state to deleting.
+            blockDataTable.putWithBatch(batch, deletingKey, blkInfo);
+            blockDataTable.deleteWithBatch(batch, blk);
             containerDB.getStore().getBatchHandler()
-                    .commitBatchOperation(batch);
+                .commitBatchOperation(batch);
             newDeletionBlocks++;
             if (LOG.isDebugEnabled()) {
               LOG.debug("Transited Block {} to DELETING state in container {}",
@@ -259,32 +259,33 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
 
       if (newDeletionBlocks > 0) {
         // Finally commit the DB counters.
-        BatchOperation batchOperation = containerDB.getStore().getBatchHandler()
-                .initBatchOperation();
-        Table<String, Long> metadataTable = containerDB.getStore()
-                .getMetadataTable();
-  
-        // In memory is updated only when existing delete transactionID is
-        // greater.
-        if (delTX.getTxID() > containerData.getDeleteTransactionId()) {
-          // Update in DB pending delete key count and delete transaction ID.
+        try(BatchOperation batchOperation =
+                containerDB.getStore().getBatchHandler().initBatchOperation()) {
+          Table< String, Long > metadataTable = containerDB.getStore()
+              .getMetadataTable();
+
+          // In memory is updated only when existing delete transactionID is
+          // greater.
+          if (delTX.getTxID() > containerData.getDeleteTransactionId()) {
+            // Update in DB pending delete key count and delete transaction ID.
+            metadataTable.putWithBatch(batchOperation,
+                OzoneConsts.DELETE_TRANSACTION_KEY, delTX.getTxID());
+          }
+
+          long pendingDeleteBlocks =
+              containerData.getNumPendingDeletionBlocks() + newDeletionBlocks;
           metadataTable.putWithBatch(batchOperation,
-                  OzoneConsts.DELETE_TRANSACTION_KEY, delTX.getTxID());
-        }
-  
-        long pendingDeleteBlocks = containerData.getNumPendingDeletionBlocks() +
-                newDeletionBlocks;
-        metadataTable.putWithBatch(batchOperation,
-                OzoneConsts.PENDING_DELETE_BLOCK_COUNT, pendingDeleteBlocks);
-  
-        containerDB.getStore().getBatchHandler()
-                .commitBatchOperation(batchOperation);
-  
-        // update pending deletion blocks count and delete transaction ID in
-        // in-memory container status
-        containerData.updateDeleteTransactionId(delTX.getTxID());
+              OzoneConsts.PENDING_DELETE_BLOCK_COUNT, pendingDeleteBlocks);
+
+          containerDB.getStore().getBatchHandler()
+              .commitBatchOperation(batchOperation);
 
-        containerData.incrPendingDeletionBlocks(newDeletionBlocks);
+          // update pending deletion blocks count and delete transaction ID in
+          // in-memory container status
+          containerData.updateDeleteTransactionId(delTX.getTxID());
+
+          containerData.incrPendingDeletionBlocks(newDeletionBlocks);
+        }
       }
     }
   }
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 1998d0e..474092e 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
@@ -124,31 +124,32 @@ public class BlockManagerImpl implements BlockManager {
         return data.getSize();
       }
       // update the blockData as well as BlockCommitSequenceId here
-      BatchOperation batch = db.getStore().getBatchHandler()
-              .initBatchOperation();
-      db.getStore().getBlockDataTable().putWithBatch(
-              batch, Long.toString(data.getLocalID()), data);
-      db.getStore().getMetadataTable().putWithBatch(
-              batch, OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID, bcsId);
-
-      // Set Bytes used, this bytes used will be updated for every write and
-      // only get committed for every put block. In this way, when datanode
-      // is up, for computation of disk space by container only committed
-      // block length is used, And also on restart the blocks committed to DB
-      // 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());
-
-      // Set Block Count for a container.
-      if (incrKeyCount) {
+      try(BatchOperation batch = db.getStore().getBatchHandler()
+              .initBatchOperation()) {
+        db.getStore().getBlockDataTable().putWithBatch(
+            batch, Long.toString(data.getLocalID()), data);
         db.getStore().getMetadataTable().putWithBatch(
-                batch, OzoneConsts.BLOCK_COUNT,
-                container.getContainerData().getKeyCount() + 1);
-      }
+            batch, OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID, bcsId);
+
+        // Set Bytes used, this bytes used will be updated for every write and
+        // only get committed for every put block. In this way, when datanode
+        // is up, for computation of disk space by container only committed
+        // block length is used, And also on restart the blocks committed to DB
+        // 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());
+
+        // Set Block Count for a container.
+        if (incrKeyCount) {
+          db.getStore().getMetadataTable().putWithBatch(
+              batch, OzoneConsts.BLOCK_COUNT,
+              container.getContainerData().getKeyCount() + 1);
+        }
 
-      db.getStore().getBatchHandler().commitBatchOperation(batch);
+        db.getStore().getBatchHandler().commitBatchOperation(batch);
+      }
 
       container.updateBlockCommitSequenceId(bcsId);
       // Increment block count finally here for in-memory.
@@ -258,17 +259,18 @@ public class BlockManagerImpl implements BlockManager {
       getBlockByID(db, blockID);
 
       // Update DB to delete block and set block count and bytes used.
-      BatchOperation batch = db.getStore().getBatchHandler()
-              .initBatchOperation();
-      String localID = Long.toString(blockID.getLocalID());
-      db.getStore().getBlockDataTable().deleteWithBatch(batch, localID);
-      // Update DB to delete block and set block count.
-      // No need to set bytes used here, as bytes used is taken care during
-      // delete chunk.
-      long blockCount = container.getContainerData().getKeyCount() - 1;
-      db.getStore().getMetadataTable()
-              .putWithBatch(batch, OzoneConsts.BLOCK_COUNT, blockCount);
-      db.getStore().getBatchHandler().commitBatchOperation(batch);
+      try(BatchOperation batch = db.getStore().getBatchHandler()
+              .initBatchOperation()) {
+        String localID = Long.toString(blockID.getLocalID());
+        db.getStore().getBlockDataTable().deleteWithBatch(batch, localID);
+        // Update DB to delete block and set block count.
+        // No need to set bytes used here, as bytes used is taken care during
+        // delete chunk.
+        long blockCount = container.getContainerData().getKeyCount() - 1;
+        db.getStore().getMetadataTable()
+            .putWithBatch(batch, OzoneConsts.BLOCK_COUNT, blockCount);
+        db.getStore().getBatchHandler().commitBatchOperation(batch);
+      }
 
       // Decrement block count here
       container.getContainerData().decrKeyCount();
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 84334de..a373c21 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
@@ -295,29 +295,32 @@ public class BlockDeletingService extends BackgroundService {
 
         // Once files are deleted... replace deleting entries with deleted
         // entries
-        BatchOperation batch = meta.getStore().getBatchHandler()
-                .initBatchOperation();
-        Table<String, ChunkInfoList> deletedBlocksTable =
-                meta.getStore().getDeletedBlocksTable();
-        for (String entry: succeedBlocks) {
-          List<ContainerProtos.ChunkInfo> chunkList =
-                  blockDataTable.get(entry).getChunks();
-          String blockId = entry.substring(
-                      OzoneConsts.DELETING_KEY_PREFIX.length());
-
-          deletedBlocksTable.putWithBatch(
-                  batch, blockId,
-                  new ChunkInfoList(chunkList));
-          blockDataTable.deleteWithBatch(batch, entry);
-        }
+        try(BatchOperation batch = meta.getStore().getBatchHandler()
+                .initBatchOperation()) {
+          Table< String, ChunkInfoList > deletedBlocksTable =
+              meta.getStore().getDeletedBlocksTable();
+          for (String entry : succeedBlocks) {
+            List< ContainerProtos.ChunkInfo > chunkList =
+                blockDataTable.get(entry).getChunks();
+            String blockId = entry.substring(
+                OzoneConsts.DELETING_KEY_PREFIX.length());
+
+            deletedBlocksTable.putWithBatch(
+                batch, blockId,
+                new ChunkInfoList(chunkList));
+            blockDataTable.deleteWithBatch(batch, entry);
+          }
 
-        int deleteBlockCount = succeedBlocks.size();
-        containerData.updateAndCommitDBCounters(meta, batch, deleteBlockCount);
+          int deleteBlockCount = succeedBlocks.size();
+          containerData.updateAndCommitDBCounters(meta, batch,
+              deleteBlockCount);
 
-        // update count of pending deletion blocks and block count in in-memory
-        // container status.
-        containerData.decrPendingDeletionBlocks(deleteBlockCount);
-        containerData.decrKeyCount(deleteBlockCount);
+
+          // update count of pending deletion blocks and block count in
+          // in-memory container status.
+          containerData.decrPendingDeletionBlocks(deleteBlockCount);
+          containerData.decrKeyCount(deleteBlockCount);
+        }
 
         if (!succeedBlocks.isEmpty()) {
           LOG.info("Container: {}, deleted blocks: {}, task elapsed time: {}ms",
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
index acc516a..edd3d4a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
@@ -302,15 +302,18 @@ public class DeletedBlockLogImpl
       throws IOException {
     lock.lock();
     try {
-      BatchOperation batch = scmMetadataStore.getStore().initBatchOperation();
-      for (Map.Entry<Long, List<Long>> entry : containerBlocksMap.entrySet()) {
-        long nextTXID = scmMetadataStore.getNextDeleteBlockTXID();
-        DeletedBlocksTransaction tx = constructNewTransaction(nextTXID,
-            entry.getKey(), entry.getValue());
-        scmMetadataStore.getDeletedBlocksTXTable().putWithBatch(batch,
-            nextTXID, tx);
+      try(BatchOperation batch =
+          scmMetadataStore.getStore().initBatchOperation()) {
+        for (Map.Entry< Long, List< Long > > entry :
+            containerBlocksMap.entrySet()) {
+          long nextTXID = scmMetadataStore.getNextDeleteBlockTXID();
+          DeletedBlocksTransaction tx = constructNewTransaction(nextTXID,
+              entry.getKey(), entry.getValue());
+          scmMetadataStore.getDeletedBlocksTXTable().putWithBatch(batch,
+              nextTXID, tx);
+        }
+        scmMetadataStore.getStore().commitBatchOperation(batch);
       }
-      scmMetadataStore.getStore().commitBatchOperation(batch);
     } finally {
       lock.unlock();
     }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/SCMContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/SCMContainerManager.java
index d56c47e..19a5ab2 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/SCMContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/SCMContainerManager.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.hdds.utils.db.BatchOperationHandler;
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.hdds.utils.db.Table.KeyValue;
@@ -51,6 +52,8 @@ import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER;
+
 /**
  * ContainerManager class contains the mapping from a name to a pipeline
  * mapping. This is used by SCM when allocating new locations and when
@@ -322,7 +325,7 @@ public class SCMContainerManager implements ContainerManager {
       throw new SCMException(
           "Failed to delete container " + containerID + ", reason : " +
               "container doesn't exist.",
-          SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
+          FAILED_TO_FIND_CONTAINER);
     } finally {
       lock.unlock();
     }
@@ -378,7 +381,7 @@ public class SCMContainerManager implements ContainerManager {
           "Failed to update container state"
               + containerID
               + ", reason : container doesn't exist.",
-          SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
+          FAILED_TO_FIND_CONTAINER);
     } finally {
       lock.unlock();
     }
@@ -397,33 +400,28 @@ public class SCMContainerManager implements ContainerManager {
     if (deleteTransactionMap == null) {
       return;
     }
-    org.apache.hadoop.hdds.utils.db.BatchOperation batchOperation =
-        batchHandler.initBatchOperation();
     lock.lock();
-    try {
-      for (Map.Entry<Long, Long> entry : deleteTransactionMap.entrySet()) {
+    try(BatchOperation batchOperation = batchHandler.initBatchOperation()) {
+      for (Map.Entry< Long, Long > entry : deleteTransactionMap.entrySet()) {
         long containerID = entry.getKey();
-
         ContainerID containerIdObject = new ContainerID(containerID);
         ContainerInfo containerInfo =
             containerStore.get(containerIdObject);
         ContainerInfo containerInfoInMem = containerStateManager
             .getContainer(containerIdObject);
         if (containerInfo == null || containerInfoInMem == null) {
-          throw new SCMException(
-              "Failed to increment number of deleted blocks for container "
-                  + containerID + ", reason : " + "container doesn't exist.",
-              SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
+          throw new SCMException("Failed to increment number of deleted " +
+              "blocks for container " + containerID + ", reason : " +
+              "container doesn't exist.", FAILED_TO_FIND_CONTAINER);
         }
         containerInfo.updateDeleteTransactionId(entry.getValue());
         containerInfo.setNumberOfKeys(containerInfoInMem.getNumberOfKeys());
         containerInfo.setUsedBytes(containerInfoInMem.getUsedBytes());
-        containerStore
-            .putWithBatch(batchOperation, containerIdObject, containerInfo);
+        containerStore.putWithBatch(batchOperation, containerIdObject,
+            containerInfo);
       }
       batchHandler.commitBatchOperation(batchOperation);
-      containerStateManager
-          .updateDeleteTransactionId(deleteTransactionMap);
+      containerStateManager.updateDeleteTransactionId(deleteTransactionMap);
     } finally {
       lock.unlock();
     }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 212a8e1..e58af8b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -3601,15 +3601,16 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
 
 
       // Commit to DB.
-      BatchOperation batchOperation =
-          metadataManager.getStore().initBatchOperation();
+      try(BatchOperation batchOperation =
+          metadataManager.getStore().initBatchOperation()) {
+        metadataManager.getVolumeTable().putWithBatch(batchOperation,
+            dbVolumeKey, omVolumeArgs);
 
-      metadataManager.getVolumeTable().putWithBatch(batchOperation, dbVolumeKey,
-          omVolumeArgs);
-      metadataManager.getUserTable().putWithBatch(batchOperation, dbUserKey,
-          userVolumeInfo);
+        metadataManager.getUserTable().putWithBatch(batchOperation, dbUserKey,
+            userVolumeInfo);
 
-      metadataManager.getStore().commitBatchOperation(batchOperation);
+        metadataManager.getStore().commitBatchOperation(batchOperation);
+      }
 
       // Add to cache.
       metadataManager.getVolumeTable().addCacheEntry(
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java
index 31e6975..c3e2d03 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java
@@ -137,15 +137,16 @@ public class TestOMKeyPurgeRequestAndResponse extends TestOMKeyRequest {
         .setStatus(Status.OK)
         .build();
 
-    BatchOperation batchOperation =
-        omMetadataManager.getStore().initBatchOperation();
+    try(BatchOperation batchOperation =
+        omMetadataManager.getStore().initBatchOperation()) {
 
-    OMKeyPurgeResponse omKeyPurgeResponse = new OMKeyPurgeResponse(
-        omResponse, deletedKeyNames);
-    omKeyPurgeResponse.addToDBBatch(omMetadataManager, batchOperation);
+      OMKeyPurgeResponse omKeyPurgeResponse = new OMKeyPurgeResponse(
+          omResponse, deletedKeyNames);
+      omKeyPurgeResponse.addToDBBatch(omMetadataManager, batchOperation);
 
-    // Do manual commit and see whether addToBatch is successful or not.
-    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+      // Do manual commit and see whether addToBatch is successful or not.
+      omMetadataManager.getStore().commitBatchOperation(batchOperation);
+    }
 
     // The keys should not exist in the DeletedKeys table
     for (String deletedKey : deletedKeyNames) {
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketCreateResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketCreateResponse.java
index 42341e3..3e71cba 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketCreateResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketCreateResponse.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.ozone.om.response.bucket;
 import java.util.UUID;
 
 import org.apache.hadoop.hdds.utils.db.Table;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -61,6 +62,13 @@ public class TestOMBucketCreateResponse {
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
 
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
+
   @Test
   public void testAddToDBBatch() throws Exception {
     String volumeName = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketDeleteResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketDeleteResponse.java
index 5331c5e..838f2b2 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketDeleteResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketDeleteResponse.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.om.response.bucket;
 
 import java.util.UUID;
 
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -63,6 +64,13 @@ public class TestOMBucketDeleteResponse {
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
 
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
+
   @Test
   public void testAddToDBBatch() throws Exception {
     String volumeName = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketSetPropertyResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketSetPropertyResponse.java
index cdf2a59..d0f4e21 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketSetPropertyResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketSetPropertyResponse.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.ozone.om.response.bucket;
 import java.util.UUID;
 
 import org.apache.hadoop.hdds.utils.db.Table;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -61,6 +62,14 @@ public class TestOMBucketSetPropertyResponse {
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
 
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
+
+
   @Test
   public void testAddToDBBatch() throws Exception {
     String volumeName = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponse.java
index fbd3af0..dc00f3e 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponse.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .OMResponse;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -59,6 +60,13 @@ public class TestOMDirectoryCreateResponse {
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
 
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
+
   @Test
   public void testAddToDBBatch() throws Exception {
 
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java
index 626a3de..1ad4c70 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java
@@ -70,6 +70,9 @@ public class TestOMKeyResponse {
   @After
   public void stop() {
     Mockito.framework().clearInlineMocks();
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
   }
 
 }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3MultipartResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3MultipartResponse.java
index ca525d2..d185d0b 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3MultipartResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3MultipartResponse.java
@@ -25,6 +25,7 @@ import java.util.UUID;
 
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
@@ -71,6 +72,14 @@ public class TestS3MultipartResponse {
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
 
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
+
+
 
   public S3InitiateMultipartUploadResponse createS3InitiateMPUResponse(
       String volumeName, String bucketName, String keyName,
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/security/TestOMDelegationTokenResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/security/TestOMDelegationTokenResponse.java
index 816e696..b35c335 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/security/TestOMDelegationTokenResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/security/TestOMDelegationTokenResponse.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
@@ -48,4 +49,11 @@ public class TestOMDelegationTokenResponse {
     omMetadataManager = new OmMetadataManagerImpl((OzoneConfiguration) conf);
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
+
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java
index 98b29ba..6c93d37 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .UserVolumeInfo;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -63,6 +64,13 @@ public class TestOMVolumeCreateResponse {
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
 
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
+
   @Test
   public void testAddToDBBatch() throws Exception {
 
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java
index 1945bb3..ae8a8c8 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .UserVolumeInfo;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -63,6 +64,13 @@ public class TestOMVolumeDeleteResponse {
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
 
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
+
   @Test
   public void testAddToDBBatch() throws Exception {
 
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java
index 0b7bf76..b6f6335 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.hdds.utils.db.Table;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -64,6 +65,14 @@ public class TestOMVolumeSetOwnerResponse {
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
 
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
+
+
   @Test
   public void testAddToDBBatch() throws Exception {
 
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetQuotaResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetQuotaResponse.java
index 9eb86c4..dfbaeac 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetQuotaResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetQuotaResponse.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.hdds.utils.db.Table;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -62,6 +63,14 @@ public class TestOMVolumeSetQuotaResponse {
     batchOperation = omMetadataManager.getStore().initBatchOperation();
   }
 
+  @After
+  public void tearDown() {
+    if (batchOperation != null) {
+      batchOperation.close();
+    }
+  }
+
+
   @Test
   public void testAddToDBBatch() throws Exception {
 


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