You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ra...@apache.org on 2022/04/05 03:17:07 UTC

[ozone] branch master updated: HDDS-6435. Add read lock waiting and held time metrics (#3180)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c23aae0d93 HDDS-6435. Add read lock waiting and held time metrics (#3180)
c23aae0d93 is described below

commit c23aae0d9384a6ad815dc3ad9edeb3cd935c53a5
Author: tanvipenumudy <46...@users.noreply.github.com>
AuthorDate: Tue Apr 5 08:47:02 2022 +0530

    HDDS-6435. Add read lock waiting and held time metrics (#3180)
---
 .../org/apache/hadoop/ozone/lock/ActiveLock.java   |  14 ++-
 .../org/apache/hadoop/ozone/lock/LockManager.java  |  15 +++
 .../apache/hadoop/ozone/om/lock/LockUsageInfo.java |  44 +++++++
 .../apache/hadoop/ozone/om/lock/OMLockMetrics.java | 139 +++++++++++++++++++++
 .../hadoop/ozone/om/lock/OzoneManagerLock.java     | 139 ++++++++++++++++++++-
 .../hadoop/ozone/om/lock/TestOzoneManagerLock.java |  89 +++++++++++++
 .../hadoop/ozone/om/OmMetadataManagerImpl.java     |   3 +-
 7 files changed, 437 insertions(+), 6 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java
index 95dfd6c393..7a9a56769b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.ozone.lock;
 
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 /**
@@ -26,7 +25,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
  */
 public final class ActiveLock {
 
-  private ReadWriteLock lock;
+  private ReentrantReadWriteLock lock;
   private AtomicInteger count;
 
   /**
@@ -127,6 +126,17 @@ public final class ActiveLock {
     return count.get();
   }
 
+  /**
+   * Returns the number of reentrant read holds on this lock by the current
+   * thread.
+   *
+   * @return the number of holds on the read lock by the current thread,
+   *         or zero if the read lock is not held by the current thread
+   */
+  int getReadHoldCount() {
+    return lock.getReadHoldCount();
+  }
+
   /**
    * Resets the active count on the lock.
    */
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/LockManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/LockManager.java
index ac6e9a13ab..f0b458ec47 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/LockManager.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/LockManager.java
@@ -234,4 +234,19 @@ public class LockManager<R> {
     });
   }
 
+  /**
+   * Returns the number of reentrant read holds on this lock by the current
+   * thread on a given resource.
+   *
+   * @param resource for which the read lock hold count has to be returned
+   * @return the number of holds on the read lock by the current thread,
+   *         or zero if the read lock is not held by the current thread
+   */
+  public int getReadHoldCount(final R resource) {
+    ActiveLock activeLock = activeLocks.get(resource);
+    if (activeLock != null) {
+      return activeLock.getReadHoldCount();
+    }
+    return 0;
+  }
 }
\ No newline at end of file
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/LockUsageInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/LockUsageInfo.java
new file mode 100644
index 0000000000..b27f291215
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/LockUsageInfo.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om.lock;
+
+/**
+ * Maintains lock related information useful in updating OMLockMetrics.
+ */
+public class LockUsageInfo {
+
+  private long startHeldTimeNanos = -1;
+
+  /**
+   * Sets the time (ns) when the lock holding period begins.
+   *
+   * @param startLockHeldTimeNanos lock held start time (ns)
+   */
+  public void setStartHeldTimeNanos(long startLockHeldTimeNanos) {
+    this.startHeldTimeNanos = startLockHeldTimeNanos;
+  }
+
+  /**
+   * Returns the time (ns) when the lock holding period began.
+   *
+   * @return lock held start time (ns)
+   */
+  public long getStartHeldTimeNanos() {
+    return startHeldTimeNanos;
+  }
+};
\ No newline at end of file
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OMLockMetrics.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OMLockMetrics.java
new file mode 100644
index 0000000000..c8490502b1
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OMLockMetrics.java
@@ -0,0 +1,139 @@
+/**
+ * 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.om.lock;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableStat;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * This class is for maintaining the various Ozone Manager Lock Metrics.
+ */
+@InterfaceAudience.Private
+@Metrics(about = "Ozone Manager Lock Metrics", context = OzoneConsts.OZONE)
+public final class OMLockMetrics implements MetricsSource {
+  private static final String SOURCE_NAME =
+      OMLockMetrics.class.getSimpleName();
+
+  private final MetricsRegistry registry;
+  private final MutableStat readLockWaitingTimeMsStat;
+  private final MutableStat readLockHeldTimeMsStat;
+
+  private OMLockMetrics() {
+    registry = new MetricsRegistry(SOURCE_NAME);
+    readLockWaitingTimeMsStat = registry.newStat("ReadLockWaitTime",
+        "Time (in milliseconds) spent waiting for acquiring the lock",
+        "Ops", "Time", true);
+    readLockHeldTimeMsStat = registry.newStat("ReadLockHeldTime",
+        "Time (in milliseconds) spent holding the lock",
+        "Ops", "Time", true);
+  }
+
+  /**
+   * Registers OMLockMetrics source.
+   *
+   * @return OMLockMetrics object
+   */
+  public static OMLockMetrics create() {
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    return ms.register(SOURCE_NAME, "Ozone Manager Lock Metrics",
+        new OMLockMetrics());
+  }
+
+  /**
+   * Unregisters OMLockMetrics source.
+   */
+  public void unRegister() {
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    ms.unregisterSource(SOURCE_NAME);
+  }
+
+  /**
+   * Adds a snapshot to the metric readLockWaitingTimeMsStat.
+   *
+   * @param readLockWaitingTimeMs read lock waiting time (ms)
+   */
+  public void setReadLockWaitingTimeMsStat(long readLockWaitingTimeMs) {
+    this.readLockWaitingTimeMsStat.add(readLockWaitingTimeMs);
+  }
+
+  /**
+   * Adds a snapshot to the metric readLockHeldTimeMsStat.
+   *
+   * @param readLockHeldTimeMs read lock held time (ms)
+   */
+  public void setReadLockHeldTimeMsStat(long readLockHeldTimeMs) {
+    this.readLockHeldTimeMsStat.add(readLockHeldTimeMs);
+  }
+
+  /**
+   * Returns a string representation of the object. Provides information on the
+   * total number of samples, minimum value, maximum value, arithmetic mean,
+   * standard deviation of all the samples added.
+   *
+   * @return String representation of object
+   */
+  public String getReadLockWaitingTimeMsStat() {
+    return readLockWaitingTimeMsStat.toString();
+  }
+
+  /**
+   * Returns the longest time (ms) a read lock was waiting since the last
+   * measurement.
+   *
+   * @return longest read lock waiting time (ms)
+   */
+  public long getLongestReadLockWaitingTimeMs() {
+    return (long) readLockWaitingTimeMsStat.lastStat().max();
+  }
+
+  /**
+   * Returns a string representation of the object. Provides information on the
+   * total number of samples, minimum value, maximum value, arithmetic mean,
+   * standard deviation of all the samples added.
+   *
+   * @return String representation of object
+   */
+  public String getReadLockHeldTimeMsStat() {
+    return readLockHeldTimeMsStat.toString();
+  }
+
+  /**
+   * Returns the longest time (ms) a read lock was held since the last
+   * measurement.
+   *
+   * @return longest read lock held time (ms)
+   */
+  public long getLongestReadLockHeldTimeMs() {
+    return (long) readLockHeldTimeMsStat.lastStat().max();
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    MetricsRecordBuilder builder = collector.addRecord(SOURCE_NAME);
+    readLockHeldTimeMsStat.snapshot(builder, all);
+    readLockWaitingTimeMsStat.snapshot(builder, all);
+  }
+}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java
index ca3de18d15..6fb98ece2b 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java
@@ -21,8 +21,11 @@ package org.apache.hadoop.ozone.om.lock;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,10 +86,10 @@ public class OzoneManagerLock {
   private static final String WRITE_LOCK = "write";
 
   private final LockManager<String> manager;
+  private OMLockMetrics omLockMetrics;
   private final ThreadLocal<Short> lockSet = ThreadLocal.withInitial(
       () -> Short.valueOf((short)0));
 
-
   /**
    * Creates new OzoneManagerLock instance.
    * @param conf Configuration object
@@ -95,6 +98,7 @@ public class OzoneManagerLock {
     boolean fair = conf.getBoolean(OZONE_MANAGER_FAIR_LOCK,
         OZONE_MANAGER_FAIR_LOCK_DEFAULT);
     manager = new LockManager<>(conf, fair);
+    omLockMetrics = OMLockMetrics.create();
   }
 
   /**
@@ -172,7 +176,16 @@ public class OzoneManagerLock {
       LOG.error(errorMessage);
       throw new RuntimeException(errorMessage);
     } else {
+      long startWaitingTimeNanos = Time.monotonicNowNanos();
+      /**
+       *  readHoldCount helps in metrics updation only once in case of reentrant
+       *  locks.
+       */
+      int readHoldCount = manager.getReadHoldCount(resourceName);
       lockFn.accept(resourceName);
+      if (readHoldCount == 0) {
+        updateReadLockMetrics(resource, lockType, startWaitingTimeNanos);
+      }
       if (LOG.isDebugEnabled()) {
         LOG.debug("Acquired {} {} lock on resource {}", lockType, resource.name,
             resourceName);
@@ -182,6 +195,20 @@ public class OzoneManagerLock {
     }
   }
 
+  private void updateReadLockMetrics(Resource resource, String lockType,
+                                     long startWaitingTimeNanos) {
+    if (lockType.equals(READ_LOCK)) {
+      long readLockWaitingTimeNanos =
+          Time.monotonicNowNanos() - startWaitingTimeNanos;
+
+      // Adds a snapshot to the metric readLockWaitingTimeMsStat.
+      omLockMetrics.setReadLockWaitingTimeMsStat(
+          TimeUnit.NANOSECONDS.toMillis(readLockWaitingTimeNanos));
+
+      resource.setStartHeldTimeNanos(Time.monotonicNowNanos());
+    }
+  }
+
   /**
    * Generate resource name to be locked.
    * @param resource
@@ -362,6 +389,14 @@ public class OzoneManagerLock {
     // releasing lower order level lock, as for that we need counter for
     // locks, as some locks support acquiring lock again.
     lockFn.accept(resourceName);
+    /**
+     *  readHoldCount helps in metrics updation only once in case of reentrant
+     *  locks.
+     */
+    int readHoldCount = manager.getReadHoldCount(resourceName);
+    if (readHoldCount == 0) {
+      updateReadUnlockMetrics(resource, lockType);
+    }
     // clear lock
     if (LOG.isDebugEnabled()) {
       LOG.debug("Release {} {}, lock on resource {}", lockType, resource.name,
@@ -370,6 +405,81 @@ public class OzoneManagerLock {
     lockSet.set(resource.clearLock(lockSet.get()));
   }
 
+  private void updateReadUnlockMetrics(Resource resource, String lockType) {
+    if (lockType.equals(READ_LOCK)) {
+      long readLockHeldTimeNanos =
+          Time.monotonicNowNanos() - resource.getStartHeldTimeNanos();
+
+      // Adds a snapshot to the metric readLockHeldTimeMsStat.
+      omLockMetrics.setReadLockHeldTimeMsStat(
+          TimeUnit.NANOSECONDS.toMillis(readLockHeldTimeNanos));
+    }
+  }
+
+  /**
+   * Returns readHoldCount for a given resource lock name.
+   *
+   * @param resourceName resource lock name
+   * @return readHoldCount
+   */
+  @VisibleForTesting
+  public int getReadHoldCount(String resourceName) {
+    return manager.getReadHoldCount(resourceName);
+  }
+
+  /**
+   * Returns a string representation of the object. Provides information on the
+   * total number of samples, minimum value, maximum value, arithmetic mean,
+   * standard deviation of all the samples added.
+   *
+   * @return String representation of object
+   */
+  @VisibleForTesting
+  public String getReadLockWaitingTimeMsStat() {
+    return omLockMetrics.getReadLockWaitingTimeMsStat();
+  }
+
+  /**
+   * Returns the longest time (ms) a read lock was waiting since the last
+   * measurement.
+   *
+   * @return longest read lock waiting time (ms)
+   */
+  @VisibleForTesting
+  public long getLongestReadLockWaitingTimeMs() {
+    return omLockMetrics.getLongestReadLockWaitingTimeMs();
+  }
+
+  /**
+   * Returns a string representation of the object. Provides information on the
+   * total number of samples, minimum value, maximum value, arithmetic mean,
+   * standard deviation of all the samples added.
+   *
+   * @return String representation of object
+   */
+  @VisibleForTesting
+  public String getReadLockHeldTimeMsStat() {
+    return omLockMetrics.getReadLockHeldTimeMsStat();
+  }
+
+  /**
+   * Returns the longest time (ms) a read lock was held since the last
+   * measurement.
+   *
+   * @return longest read lock held time (ms)
+   */
+  @VisibleForTesting
+  public long getLongestReadLockHeldTimeMs() {
+    return omLockMetrics.getLongestReadLockHeldTimeMs();
+  }
+
+  /**
+   * Unregisters OMLockMetrics source.
+   */
+  public void cleanup() {
+    omLockMetrics.unRegister();
+  }
+
   /**
    * Resource defined in Ozone.
    */
@@ -404,6 +514,31 @@ public class OzoneManagerLock {
     // Name of the resource.
     private String name;
 
+    // This helps in maintaining lock related variables locally confined to a
+    // given thread.
+    private final ThreadLocal<LockUsageInfo> readLockTimeStampNanos =
+        ThreadLocal.withInitial(LockUsageInfo::new);
+
+    /**
+     * Sets the time (ns) when the lock holding period begins specific to a
+     * thread.
+     *
+     * @param startHeldTimeNanos lock held start time (ns)
+     */
+    void setStartHeldTimeNanos(long startHeldTimeNanos) {
+      readLockTimeStampNanos.get().setStartHeldTimeNanos(startHeldTimeNanos);
+    }
+
+    /**
+     * Returns the time (ns) when the lock holding period began specific to a
+     * thread.
+     *
+     * @return lock held start time (ns)
+     */
+    long getStartHeldTimeNanos() {
+      return readLockTimeStampNanos.get().getStartHeldTimeNanos();
+    }
+
     Resource(byte pos, String name) {
       this.lockLevel = pos;
       this.mask = (short) (Math.pow(2, lockLevel + 1) - 1);
@@ -472,6 +607,4 @@ public class OzoneManagerLock {
       return mask;
     }
   }
-
 }
-
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/lock/TestOzoneManagerLock.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/lock/TestOzoneManagerLock.java
index de12e795a3..be975c7450 100644
--- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/lock/TestOzoneManagerLock.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/lock/TestOzoneManagerLock.java
@@ -29,6 +29,7 @@ import org.junit.Test;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 /**
@@ -213,6 +214,21 @@ public class TestOzoneManagerLock {
     }
   }
 
+  private String generateResourceLockName(OzoneManagerLock.Resource resource,
+                                          String... resources) {
+    if (resources.length == 1 &&
+        resource != OzoneManagerLock.Resource.BUCKET_LOCK) {
+      return OzoneManagerLockUtil.generateResourceLockName(resource,
+          resources[0]);
+    } else if (resources.length == 2 &&
+        resource == OzoneManagerLock.Resource.BUCKET_LOCK) {
+      return OzoneManagerLockUtil.generateBucketLockName(resources[0],
+          resources[1]);
+    } else {
+      throw new IllegalArgumentException("acquire lock is supported on single" +
+          " resource for all locks except for resource bucket");
+    }
+  }
 
   /**
    * Class used to store locked resource info.
@@ -341,4 +357,77 @@ public class TestOzoneManagerLock {
     Thread.sleep(100);
     Assert.assertTrue(gotLock.get());
   }
+
+  @Test
+  public void testReadLockHoldCount() {
+    String[] resourceName;
+    String resourceLockName;
+    for (OzoneManagerLock.Resource resource :
+        OzoneManagerLock.Resource.values()) {
+      // USER_LOCK, S3_SECRET_LOCK and PREFIX_LOCK disallow lock re-acquire by
+      // the same thread.
+      if (resource != OzoneManagerLock.Resource.USER_LOCK &&
+          resource != OzoneManagerLock.Resource.S3_SECRET_LOCK &&
+          resource != OzoneManagerLock.Resource.PREFIX_LOCK) {
+        resourceName = generateResourceName(resource);
+        resourceLockName = generateResourceLockName(resource, resourceName);
+        testReadLockHoldCountUtil(resource, resourceName, resourceLockName);
+      }
+    }
+  }
+
+  private void testReadLockHoldCountUtil(OzoneManagerLock.Resource resource,
+                                         String[] resourceName,
+                                         String resourceLockName) {
+    OzoneManagerLock lock = new OzoneManagerLock(new OzoneConfiguration());
+
+    assertEquals(0, lock.getReadHoldCount(resourceLockName));
+    lock.acquireReadLock(resource, resourceName);
+    assertEquals(1, lock.getReadHoldCount(resourceLockName));
+
+    lock.acquireReadLock(resource, resourceName);
+    assertEquals(2, lock.getReadHoldCount(resourceLockName));
+
+    lock.releaseReadLock(resource, resourceName);
+    assertEquals(1, lock.getReadHoldCount(resourceLockName));
+
+    lock.releaseReadLock(resource, resourceName);
+    assertEquals(0, lock.getReadHoldCount(resourceLockName));
+  }
+
+  @Test
+  public void testReadLockConcurrentStats() throws InterruptedException {
+    String[] resourceName;
+    for (OzoneManagerLock.Resource resource :
+        OzoneManagerLock.Resource.values()) {
+      resourceName = generateResourceName(resource);
+
+      OzoneManagerLock lock = new OzoneManagerLock(new OzoneConfiguration());
+      final int threadCount = 10;
+      Thread[] threads = new Thread[threadCount];
+
+      for (int i = 0; i < threads.length; i++) {
+        String[] finalResourceName = resourceName;
+        threads[i] = new Thread(() -> {
+          lock.acquireReadLock(resource, finalResourceName);
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException e) {
+            e.printStackTrace();
+          }
+          lock.releaseReadLock(resource, finalResourceName);
+        });
+        threads[i].start();
+      }
+
+      for (Thread t : threads) {
+        t.join();
+      }
+
+      String readHeldStat = lock.getReadLockHeldTimeMsStat();
+      Assert.assertTrue(
+          "Expected " + threadCount + " samples in " + readHeldStat,
+          readHeldStat.contains("Samples = " + threadCount));
+    }
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
index 40cbba36c0..b609b26f0d 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
@@ -224,7 +224,6 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   private Map<String, Table> tableMap = new HashMap<>();
 
   public OmMetadataManagerImpl(OzoneConfiguration conf) throws IOException {
-
     this.lock = new OzoneManagerLock(conf);
     // TODO: This is a temporary check. Once fully implemented, all OM state
     //  change should go through Ratis - be it standalone (for non-HA) or
@@ -505,6 +504,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
       store.close();
       store = null;
     }
+    // OzoneManagerLock cleanup
+    lock.cleanup();
   }
 
   /**


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