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/22 07:04:31 UTC

[ozone] branch master updated: HDDS-6583. Introduce lock strategy pattern implementations based on configuration flags (#3310)

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 03b0a61958 HDDS-6583. Introduce lock strategy pattern implementations based on configuration flags (#3310)
03b0a61958 is described below

commit 03b0a61958a75b89fda3425ba52aae1355a174bc
Author: tanvipenumudy <46...@users.noreply.github.com>
AuthorDate: Fri Apr 22 12:34:27 2022 +0530

    HDDS-6583. Introduce lock strategy pattern implementations based on configuration flags (#3310)
---
 .../common/src/main/resources/ozone-default.xml    |   9 ++
 .../org/apache/hadoop/ozone/om/OMConfigKeys.java   |   7 ++
 .../org/apache/hadoop/ozone/om/OzoneManager.java   |  17 ++++
 .../ozone/om/lock/OBSKeyPathLockStrategy.java      | 101 +++++++++++++++++++
 .../hadoop/ozone/om/lock/OzoneLockProvider.java    |  52 ++++++++++
 .../hadoop/ozone/om/lock/OzoneLockStrategy.java    |  49 ++++++++++
 .../ozone/om/lock/RegularBucketLockStrategy.java   |  82 ++++++++++++++++
 .../apache/hadoop/ozone/om/lock/package-info.java  |  23 +++++
 .../ozone/om/lock/TestOzoneLockProvider.java       | 108 +++++++++++++++++++++
 9 files changed, 448 insertions(+)

diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index aaa78591a9..b0c6e98ed3 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -2848,6 +2848,15 @@
     </description>
   </property>
 
+  <property>
+    <name>ozone.om.key.path.lock.enabled</name>
+    <tag>OZONE, OM</tag>
+    <value>false</value>
+    <description>Defaults to false. If true, the fine-grained KEY_PATH_LOCK
+      functionality is enabled. If false, it is disabled.
+    </description>
+  </property>
+
   <property>
     <name>ozone.client.key.provider.cache.expiry</name>
     <tag>OZONE, CLIENT, SECURITY</tag>
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
index 5717ca6f3c..adbc12315a 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
@@ -324,4 +324,11 @@ public final class OMConfigKeys {
   public static final int OZONE_OM_UNFLUSHED_TRANSACTION_MAX_COUNT_DEFAULT
       = 10000;
 
+  /**
+   * This configuration shall be enabled to utilize the functionality of the
+   * fine-grained KEY_PATH_LOCK.
+   */
+  public static final String OZONE_OM_KEY_PATH_LOCK_ENABLED =
+      "ozone.om.key.path.lock.enabled";
+  public static final boolean OZONE_OM_KEY_PATH_LOCK_ENABLED_DEFAULT = false;
 }
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 7731b9e6c7..b733d89cda 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
@@ -125,6 +125,7 @@ import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfoEx;
+import org.apache.hadoop.ozone.om.lock.OzoneLockProvider;
 import org.apache.hadoop.ozone.om.protocol.OMInterServiceProtocol;
 import org.apache.hadoop.ozone.om.protocol.OMConfiguration;
 import org.apache.hadoop.ozone.om.protocolPB.OMInterServiceProtocolClientSideImpl;
@@ -226,6 +227,8 @@ import static org.apache.hadoop.ozone.OzoneConsts.TRANSACTION_INFO_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KEY_PATH_LOCK_ENABLED;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KEY_PATH_LOCK_ENABLED_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_AUTH_TYPE;
@@ -377,6 +380,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   private static boolean testReloadConfigFlag = false;
   private static boolean testSecureOmFlag = false;
 
+  private OzoneLockProvider ozoneLockProvider;
+
   /**
    * OM Startup mode.
    */
@@ -1436,6 +1441,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
       bootstrap(omNodeDetails);
     }
 
+    ozoneLockProvider = new OzoneLockProvider(getKeyPathLockEnabled(),
+        getEnableFileSystemPaths());
+
     omState = State.RUNNING;
   }
 
@@ -3683,6 +3691,15 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
         OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT);
   }
 
+  public boolean getKeyPathLockEnabled() {
+    return configuration.getBoolean(OZONE_OM_KEY_PATH_LOCK_ENABLED,
+        OZONE_OM_KEY_PATH_LOCK_ENABLED_DEFAULT);
+  }
+
+  public OzoneLockProvider getOzoneLockProvider() {
+    return this.ozoneLockProvider;
+  }
+
   public ReplicationConfig getDefaultReplicationConfig() {
     final String replication = configuration.getTrimmed(
         OZONE_SERVER_DEFAULT_REPLICATION_KEY,
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OBSKeyPathLockStrategy.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OBSKeyPathLockStrategy.java
new file mode 100644
index 0000000000..dc066f5654
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OBSKeyPathLockStrategy.java
@@ -0,0 +1,101 @@
+/**
+ * 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;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Implementation of OzoneLockStrategy interface. Concrete strategy for OBS
+ * KEY_PATH_LOCK.
+ */
+public class OBSKeyPathLockStrategy implements OzoneLockStrategy {
+
+  @Override
+  public boolean acquireWriteLock(OMMetadataManager omMetadataManager,
+                                  OzoneManagerLock.Resource resource,
+                                  String volumeName, String bucketName,
+                                  String keyName) throws IOException {
+    boolean acquiredLock;
+
+    acquiredLock = omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK,
+        volumeName, bucketName);
+    OMFileRequest.validateBucket(omMetadataManager, volumeName, bucketName);
+
+    Preconditions.checkArgument(acquiredLock,
+        "BUCKET_LOCK should be acquired!");
+
+    acquiredLock = omMetadataManager.getLock()
+        .acquireWriteLock(resource, volumeName, bucketName, keyName);
+
+    return acquiredLock;
+  }
+
+  @Override
+  public void releaseWriteLock(OMMetadataManager omMetadataManager,
+                               OzoneManagerLock.Resource resource,
+                               String volumeName, String bucketName,
+                               String keyName) throws IOException {
+    omMetadataManager.getLock()
+        .releaseWriteLock(resource, volumeName, bucketName, keyName);
+
+    omMetadataManager.getLock()
+        .releaseReadLock(BUCKET_LOCK, volumeName, bucketName);
+
+    return;
+  }
+
+  @Override
+  public boolean acquireReadLock(OMMetadataManager omMetadataManager,
+                                 OzoneManagerLock.Resource resource,
+                                 String volumeName, String bucketName,
+                                 String keyName) throws IOException {
+    boolean acquiredLock;
+
+    acquiredLock = omMetadataManager.getLock()
+        .acquireReadLock(BUCKET_LOCK, volumeName, bucketName);
+    OMFileRequest.validateBucket(omMetadataManager, volumeName, bucketName);
+
+    Preconditions.checkArgument(acquiredLock,
+        "BUCKET_LOCK should be acquired!");
+
+    acquiredLock = omMetadataManager.getLock()
+        .acquireReadLock(resource, volumeName, bucketName, keyName);
+
+    return acquiredLock;
+  }
+
+  @Override
+  public void releaseReadLock(OMMetadataManager omMetadataManager,
+                              OzoneManagerLock.Resource resource,
+                              String volumeName, String bucketName,
+                              String keyName) throws IOException {
+    omMetadataManager.getLock()
+        .releaseReadLock(resource, volumeName, bucketName, keyName);
+
+    omMetadataManager.getLock()
+        .releaseReadLock(BUCKET_LOCK, volumeName, bucketName);
+
+    return;
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneLockProvider.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneLockProvider.java
new file mode 100644
index 0000000000..c34fff27da
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneLockProvider.java
@@ -0,0 +1,52 @@
+/**
+ * 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;
+
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+
+/**
+ * OzoneLockProvider class returns the appropriate lock strategy pattern
+ * implementation based on the configuration flags passed.
+ */
+public class OzoneLockProvider {
+
+  private boolean keyPathLockEnabled;
+  private boolean enableFileSystemPaths;
+
+  public OzoneLockProvider(boolean keyPathLockEnabled,
+                           boolean enableFileSystemPaths) {
+    this.keyPathLockEnabled = keyPathLockEnabled;
+    this.enableFileSystemPaths = enableFileSystemPaths;
+  }
+
+  public OzoneLockStrategy createLockStrategy(BucketLayout bucketLayout) {
+
+    // TODO: This can be extended to support FSO, LEGACY_FS in the future.
+    if (keyPathLockEnabled) {
+      if (bucketLayout == BucketLayout.OBJECT_STORE) {
+        return new OBSKeyPathLockStrategy();
+      } else if (!enableFileSystemPaths &&
+          bucketLayout == BucketLayout.LEGACY) {
+        // old pre-created bucket with enableFileSystemPaths = false.
+        return new OBSKeyPathLockStrategy();
+      }
+    }
+
+    return new RegularBucketLockStrategy();
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneLockStrategy.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneLockStrategy.java
new file mode 100644
index 0000000000..951e4a5ab4
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneLockStrategy.java
@@ -0,0 +1,49 @@
+/**
+ * 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;
+
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+
+import java.io.IOException;
+
+/**
+ * This is a common strategy interface for all concrete lock strategies. The
+ * interface declares methods (for acquiring/releasing a read/write lock) the
+ * context (here OzoneManager class) uses to execute a lock strategy. The
+ * concrete lock strategies (which implement different variations of the
+ * methods defined in the interface) allow us in changing the behavior of the
+ * locking mechanism at runtime.
+ */
+public interface OzoneLockStrategy {
+  boolean acquireWriteLock(OMMetadataManager omMetadataManager,
+                           OzoneManagerLock.Resource resource,
+                           String volumeName, String bucketName, String keyName)
+      throws IOException;
+
+  void releaseWriteLock(OMMetadataManager omMetadataManager,
+                        OzoneManagerLock.Resource resource, String volumeName,
+                        String bucketName, String keyName) throws IOException;
+
+  boolean acquireReadLock(OMMetadataManager omMetadataManager,
+                          OzoneManagerLock.Resource resource, String volumeName,
+                          String bucketName, String keyName) throws IOException;
+
+  void releaseReadLock(OMMetadataManager omMetadataManager,
+                       OzoneManagerLock.Resource resource, String volumeName,
+                       String bucketName, String keyName) throws IOException;
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/RegularBucketLockStrategy.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/RegularBucketLockStrategy.java
new file mode 100644
index 0000000000..161bcec7f6
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/RegularBucketLockStrategy.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <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;
+
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Implementation of OzoneLockStrategy interface. Concrete strategy for regular
+ * BUCKET_LOCK.
+ */
+public class RegularBucketLockStrategy implements OzoneLockStrategy {
+
+  @Override
+  public boolean acquireWriteLock(OMMetadataManager omMetadataManager,
+                                  OzoneManagerLock.Resource resource,
+                                  String volumeName, String bucketName,
+                                  String keyName) throws IOException {
+    boolean acquiredLock;
+
+    acquiredLock = omMetadataManager.getLock()
+        .acquireWriteLock(BUCKET_LOCK, volumeName, bucketName);
+    OMFileRequest.validateBucket(omMetadataManager, volumeName, bucketName);
+
+    return acquiredLock;
+  }
+
+  @Override
+  public void releaseWriteLock(OMMetadataManager omMetadataManager,
+                               OzoneManagerLock.Resource resource,
+                               String volumeName, String bucketName,
+                               String keyName) throws IOException {
+    omMetadataManager.getLock()
+        .releaseWriteLock(BUCKET_LOCK, volumeName, bucketName);
+
+    return;
+  }
+
+  @Override
+  public boolean acquireReadLock(OMMetadataManager omMetadataManager,
+                                 OzoneManagerLock.Resource resource,
+                                 String volumeName, String bucketName,
+                                 String keyName) throws IOException {
+    boolean acquiredLock;
+
+    acquiredLock = omMetadataManager.getLock()
+        .acquireReadLock(BUCKET_LOCK, volumeName, bucketName);
+    OMFileRequest.validateBucket(omMetadataManager, volumeName, bucketName);
+
+    return acquiredLock;
+  }
+
+  @Override
+  public void releaseReadLock(OMMetadataManager omMetadataManager,
+                              OzoneManagerLock.Resource resource,
+                              String volumeName, String bucketName,
+                              String keyName) throws IOException {
+    omMetadataManager.getLock()
+        .releaseReadLock(BUCKET_LOCK, volumeName, bucketName);
+
+    return;
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/package-info.java
new file mode 100644
index 0000000000..19158a4d53
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * 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;
+
+/**
+ * This package contains classes and interfaces for OM Lock Strategy Patterns.
+ */
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/lock/TestOzoneLockProvider.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/lock/TestOzoneLockProvider.java
new file mode 100644
index 0000000000..77c7596b65
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/lock/TestOzoneLockProvider.java
@@ -0,0 +1,108 @@
+/**
+ * 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;
+
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.mockito.Mockito.when;
+
+/**
+ * Test for OzoneLockProvider.
+ */
+@RunWith(Parameterized.class)
+public class TestOzoneLockProvider {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOzoneLockProvider.class);
+
+  private OzoneManager ozoneManager;
+  private OzoneLockStrategy ozoneLockStrategy;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(
+        new Object[]{true, true},
+        new Object[]{true, false},
+        new Object[]{false, true},
+        new Object[]{false, false});
+  }
+
+  public TestOzoneLockProvider(boolean setKeyPathLock,
+                               boolean setFileSystemPaths) {
+    // Ignored. Actual init done in initParam().
+    // This empty constructor is still required to avoid argument exception.
+  }
+
+  @Parameterized.BeforeParam
+  public static void initParam(boolean setKeyPathLock,
+                               boolean setFileSystemPaths) {
+    keyPathLockEnabled = setKeyPathLock;
+    enableFileSystemPaths = setFileSystemPaths;
+  }
+
+  private static boolean keyPathLockEnabled;
+  private static boolean enableFileSystemPaths;
+
+  @Before
+  public void setup() throws Exception {
+    ozoneManager = Mockito.mock(OzoneManager.class);
+  }
+
+  @Test
+  public void testOzoneLockProvider() {
+    for (BucketLayout bucketLayout : BucketLayout.values()) {
+      testOzoneLockProviderUtil(bucketLayout);
+    }
+  }
+
+  public void testOzoneLockProviderUtil(BucketLayout bucketLayout) {
+
+    LOG.info("keyPathLockEnabled: " + keyPathLockEnabled);
+    LOG.info("enableFileSystemPaths: " + enableFileSystemPaths);
+    LOG.info("bucketLayout: " + bucketLayout + "\n");
+
+    when(ozoneManager.getOzoneLockProvider()).thenReturn(
+        new OzoneLockProvider(keyPathLockEnabled, enableFileSystemPaths));
+    ozoneLockStrategy =
+        ozoneManager.getOzoneLockProvider().createLockStrategy(bucketLayout);
+
+    if (keyPathLockEnabled) {
+      if (bucketLayout == BucketLayout.OBJECT_STORE) {
+        Assert.assertTrue(ozoneLockStrategy instanceof OBSKeyPathLockStrategy);
+      } else if (!enableFileSystemPaths &&
+          bucketLayout == BucketLayout.LEGACY) {
+        Assert.assertTrue(ozoneLockStrategy instanceof OBSKeyPathLockStrategy);
+      }
+    } else {
+      Assert.assertTrue(ozoneLockStrategy instanceof RegularBucketLockStrategy);
+    }
+  }
+}


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