szetszwo commented on code in PR #8217:
URL: https://github.com/apache/ozone/pull/8217#discussion_r2029294980


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OmLockInfo.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      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 java.util.HashSet;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * Lock information.
+ */
+public final class OmLockInfo {
+  private final LockInfo volumeLock;
+  private final LockInfo bucketLock;
+  private final Set<LockInfo> keyLocks;
+
+  private OmLockInfo(Builder builder) {
+    volumeLock = builder.volumeLock;
+    bucketLock = builder.bucketLock;
+    keyLocks = builder.keyLocks;
+  }
+
+  public Optional<LockInfo> getVolumeLock() {
+    return Optional.ofNullable(volumeLock);
+  }
+
+  public Optional<LockInfo> getBucketLock() {
+    return Optional.ofNullable(bucketLock);
+  }
+
+  public Optional<Set<LockInfo>> getKeyLocks() {
+    return Optional.ofNullable(keyLocks);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    if (volumeLock != null) {
+      sb.append("Volume:").append(volumeLock);
+    }
+    if (bucketLock != null) {
+      sb.append("Bucket:").append(bucketLock);
+    }
+    if (keyLocks != null) {
+      sb.append("Keys:").append(keyLocks);
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Builds an {@link OmLockInfo} object with optional volume, bucket or key 
locks.
+   */
+  public static final class Builder {
+    private LockInfo volumeLock;
+    private LockInfo bucketLock;
+    private Set<LockInfo> keyLocks;
+
+    public Builder() {
+    }
+
+    public Builder addVolumeReadLock(String volume) {
+      volumeLock = LockInfo.writeLockInfo(volume);
+      return this;
+    }
+
+    public Builder addVolumeWriteLock(String volume) {
+      volumeLock = LockInfo.readLockInfo(volume);
+      return this;
+    }
+
+    public Builder addBucketReadLock(String volume, String bucket) {
+      bucketLock = LockInfo.readLockInfo(joinStrings(volume, bucket));
+      return this;
+    }
+
+    public Builder addBucketWriteLock(String volume, String bucket) {
+      bucketLock = LockInfo.writeLockInfo(joinStrings(volume, bucket));
+      return this;
+    }
+
+    // Currently there is no use case for key level read locks.
+    public Builder addKeyWriteLock(String volume, String bucket, String key) {
+      // Lazy init keys.
+      if (keyLocks == null) {
+        keyLocks = new HashSet<>();
+      }
+      keyLocks.add(LockInfo.writeLockInfo(joinStrings(volume, bucket, key)));
+      return this;
+    }
+
+    private String joinStrings(String... parts) {
+      return String.join(OzoneConsts.OZONE_URI_DELIMITER, parts);
+    }
+
+    public OmLockInfo build() {
+      return new OmLockInfo(this);
+    }
+  }
+
+  /**
+   * This class provides specifications about a lock's requirements.
+   */
+  public static final class LockInfo implements Comparable<LockInfo> {
+    private final String name;
+    private final boolean isWriteLock;

Review Comment:
   Use an enum is better than a boolean; see ["Never Use Booleans for Something 
That Has Two States Now, but Might Have More 
Later"](https://protobuf.dev/best-practices/api/#never-use-booleans-for-two-states).



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OmRequestGatekeeper.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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 com.google.common.util.concurrent.Striped;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.Stack;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import org.apache.hadoop.hdds.utils.SimpleStriped;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.util.Time;
+
+/**
+ * Manage locking of volume, bucket, keys and others.
+ */
+public class OmRequestGatekeeper {
+  private static final long LOCK_TIMEOUT_DEFAULT = 10 * 60 * 1000;
+  private static final int NUM_VOLUME_STRIPES = 1024;
+  private static final int NUM_BUCKET_STRIPES = 1024;
+  private static final int NUM_KEY_STRIPES = 4096;
+
+  private final Striped<ReadWriteLock> volumeLocks;
+  private final Striped<ReadWriteLock> bucketLocks;
+  private final Striped<ReadWriteLock> keyLocks;
+
+  public OmRequestGatekeeper() {
+    volumeLocks = SimpleStriped.readWriteLock(NUM_VOLUME_STRIPES, false);
+    bucketLocks = SimpleStriped.readWriteLock(NUM_BUCKET_STRIPES, false);
+    keyLocks = SimpleStriped.readWriteLock(NUM_KEY_STRIPES, false);
+  }
+
+  public OmLockObject lock(OmLockInfo lockInfo) throws IOException {
+    OmLockObject omLockObject = new OmLockObject(lockInfo);
+    long startTime = Time.monotonicNowNanos();
+    Optional<OmLockInfo.LockInfo> optionalVolumeLock = 
lockInfo.getVolumeLock();
+    Optional<OmLockInfo.LockInfo> optionalBucketLock = 
lockInfo.getBucketLock();
+    Optional<Set<OmLockInfo.LockInfo>> optionalKeyLocks = 
lockInfo.getKeyLocks();
+    List<Lock> locks = new ArrayList<>();
+
+    if (optionalVolumeLock.isPresent()) {
+      OmLockInfo.LockInfo volumeLockInfo = optionalVolumeLock.get();
+      if (volumeLockInfo.isWriteLock()) {
+        omLockObject.setReadStatsType(false);
+        locks.add(volumeLocks.get(volumeLockInfo.getName()).writeLock());
+      } else {
+        locks.add(volumeLocks.get(volumeLockInfo.getName()).readLock());
+      }
+    }
+
+    if (optionalBucketLock.isPresent()) {
+      OmLockInfo.LockInfo bucketLockInfo = optionalBucketLock.get();
+      if (bucketLockInfo.isWriteLock()) {
+        omLockObject.setReadStatsType(false);
+        locks.add(bucketLocks.get(bucketLockInfo.getName()).writeLock());
+      } else {
+        locks.add(bucketLocks.get(bucketLockInfo.getName()).readLock());
+      }
+    }
+
+    if (optionalKeyLocks.isPresent()) {
+      for (ReadWriteLock keyLock: keyLocks.bulkGet(optionalKeyLocks.get())) {
+        omLockObject.setReadStatsType(false);
+        locks.add(keyLock.writeLock());
+      }
+    }
+
+    try {
+      acquireLocks(locks, omLockObject.getLocks());
+      lockStatsBegin(omLockObject.getLockStats(), Time.monotonicNowNanos(), 
startTime);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new OMException("Waiting for locks is interrupted, " + lockInfo, 
OMException.ResultCodes.INTERNAL_ERROR);
+    } catch (TimeoutException e) {
+      throw new OMException("Timeout occurred for locks " + lockInfo, 
OMException.ResultCodes.TIMEOUT);
+    }
+    return omLockObject;
+  }
+
+  private void acquireLocks(List<Lock> locks, Stack<Lock> acquiredLocks) 
throws TimeoutException, InterruptedException {

Review Comment:
   Concurrently, if the `lock.tryLock(..)` is interrupted, the partial lock 
won't be released.  Getting everything right with timeout is not easy.
   
   If we are adding timeout support, we have to update the design doc first.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/OmRequestGatekeeper.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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 com.google.common.util.concurrent.Striped;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.Stack;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import org.apache.hadoop.hdds.utils.SimpleStriped;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.util.Time;
+
+/**
+ * Manage locking of volume, bucket, keys and others.
+ */
+public class OmRequestGatekeeper {
+  private static final long LOCK_TIMEOUT_DEFAULT = 10 * 60 * 1000;
+  private static final int NUM_VOLUME_STRIPES = 1024;
+  private static final int NUM_BUCKET_STRIPES = 1024;
+  private static final int NUM_KEY_STRIPES = 4096;
+
+  private final Striped<ReadWriteLock> volumeLocks;
+  private final Striped<ReadWriteLock> bucketLocks;
+  private final Striped<ReadWriteLock> keyLocks;
+
+  public OmRequestGatekeeper() {
+    volumeLocks = SimpleStriped.readWriteLock(NUM_VOLUME_STRIPES, false);
+    bucketLocks = SimpleStriped.readWriteLock(NUM_BUCKET_STRIPES, false);
+    keyLocks = SimpleStriped.readWriteLock(NUM_KEY_STRIPES, false);
+  }
+
+  public OmLockObject lock(OmLockInfo lockInfo) throws IOException {
+    OmLockObject omLockObject = new OmLockObject(lockInfo);
+    long startTime = Time.monotonicNowNanos();
+    Optional<OmLockInfo.LockInfo> optionalVolumeLock = 
lockInfo.getVolumeLock();
+    Optional<OmLockInfo.LockInfo> optionalBucketLock = 
lockInfo.getBucketLock();
+    Optional<Set<OmLockInfo.LockInfo>> optionalKeyLocks = 
lockInfo.getKeyLocks();
+    List<Lock> locks = new ArrayList<>();
+
+    if (optionalVolumeLock.isPresent()) {
+      OmLockInfo.LockInfo volumeLockInfo = optionalVolumeLock.get();
+      if (volumeLockInfo.isWriteLock()) {
+        omLockObject.setReadStatsType(false);
+        locks.add(volumeLocks.get(volumeLockInfo.getName()).writeLock());
+      } else {
+        locks.add(volumeLocks.get(volumeLockInfo.getName()).readLock());
+      }
+    }
+
+    if (optionalBucketLock.isPresent()) {
+      OmLockInfo.LockInfo bucketLockInfo = optionalBucketLock.get();
+      if (bucketLockInfo.isWriteLock()) {
+        omLockObject.setReadStatsType(false);
+        locks.add(bucketLocks.get(bucketLockInfo.getName()).writeLock());
+      } else {
+        locks.add(bucketLocks.get(bucketLockInfo.getName()).readLock());
+      }
+    }
+
+    if (optionalKeyLocks.isPresent()) {
+      for (ReadWriteLock keyLock: keyLocks.bulkGet(optionalKeyLocks.get())) {
+        omLockObject.setReadStatsType(false);
+        locks.add(keyLock.writeLock());
+      }
+    }
+
+    try {
+      acquireLocks(locks, omLockObject.getLocks());
+      lockStatsBegin(omLockObject.getLockStats(), Time.monotonicNowNanos(), 
startTime);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new OMException("Waiting for locks is interrupted, " + lockInfo, 
OMException.ResultCodes.INTERNAL_ERROR);
+    } catch (TimeoutException e) {
+      throw new OMException("Timeout occurred for locks " + lockInfo, 
OMException.ResultCodes.TIMEOUT);
+    }
+    return omLockObject;
+  }

Review Comment:
   The [lock 
overhead](https://en.wikipedia.org/wiki/Lock_(computer_science)#Granularity) 
probably is quite high in this method.  It may not be able to improve the 
performance much.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to