You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by jx...@apache.org on 2020/04/23 20:02:51 UTC

[helix] 16/20: Created LockScope interface

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

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

commit ec3fe96422caa31198ff5e5377ccedf6220bcd7a
Author: Molly Gao <mg...@mgao-mn1.linkedin.biz>
AuthorDate: Fri Feb 14 13:28:07 2020 -0800

    Created LockScope interface
---
 .../main/java/org/apache/helix/lock/LockInfo.java  | 15 +++++--
 .../main/java/org/apache/helix/lock/LockScope.java | 25 ++++++++++++
 .../helix/lock/{ => helix}/HelixLockScope.java     | 47 +++++++++++-----------
 .../lock/{ => helix}/ZKHelixNonblockingLock.java   | 19 ++++-----
 .../{ => helix}/TestZKHelixNonblockingLock.java    |  5 ++-
 5 files changed, 72 insertions(+), 39 deletions(-)

diff --git a/helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java b/helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
index 3829d40..212c93c 100644
--- a/helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
+++ b/helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
@@ -138,7 +138,7 @@ public class LockInfo extends HelixProperty {
    * Get the value for OWNER attribute of the lock from a ZNRecord
    * @return the owner id of the lock, empty string if there is no owner id set
    */
-  static String getOwner(ZNRecord znRecord) {
+  public static String getOwner(ZNRecord znRecord) {
     if (znRecord == null) {
       return DEFAULT_OWNER_TEXT;
     }
@@ -150,7 +150,7 @@ public class LockInfo extends HelixProperty {
    * Get the value for MESSAGE attribute of the lock from a ZNRecord
    * @return the message of the lock, empty string if there is no message set
    */
-  static String getMessage(ZNRecord znRecord) {
+  public static String getMessage(ZNRecord znRecord) {
     if (znRecord == null) {
       return DEFAULT_MESSAGE_TEXT;
     }
@@ -162,10 +162,19 @@ public class LockInfo extends HelixProperty {
    * Get the value for TIMEOUT attribute of the lock from a ZNRecord
    * @return the expiring time of the lock, -1 if there is no timeout set
    */
-  static long getTimeout(ZNRecord znRecord) {
+  public static long getTimeout(ZNRecord znRecord) {
     if (znRecord == null) {
       return DEFAULT_TIMEOUT_LONG;
     }
     return znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
   }
+
+  /**
+   * Check if the lock has a owner id set
+   * @return true if an owner id is set, false if not
+   */
+  public static boolean ownerIdSet(ZNRecord znRecord) {
+    String ownerId = getOwner(znRecord);
+    return !ownerId.equals(DEFAULT_OWNER_TEXT);
+  }
 }
diff --git a/helix-lock/src/main/java/org/apache/helix/lock/LockScope.java b/helix-lock/src/main/java/org/apache/helix/lock/LockScope.java
new file mode 100644
index 0000000..ff2d329
--- /dev/null
+++ b/helix-lock/src/main/java/org/apache/helix/lock/LockScope.java
@@ -0,0 +1,25 @@
+/*
+ * 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.helix.lock;
+
+public interface LockScope {
+
+  String getPath();
+}
diff --git a/helix-lock/src/main/java/org/apache/helix/lock/HelixLockScope.java b/helix-lock/src/main/java/org/apache/helix/lock/helix/HelixLockScope.java
similarity index 80%
rename from helix-lock/src/main/java/org/apache/helix/lock/HelixLockScope.java
rename to helix-lock/src/main/java/org/apache/helix/lock/helix/HelixLockScope.java
index 3df0e7e..77907ea 100644
--- a/helix-lock/src/main/java/org/apache/helix/lock/HelixLockScope.java
+++ b/helix-lock/src/main/java/org/apache/helix/lock/helix/HelixLockScope.java
@@ -17,17 +17,18 @@
  * under the License.
  */
 
-package org.apache.helix.lock;
+package org.apache.helix.lock.helix;
 
 import java.util.List;
 
+import org.apache.helix.lock.LockScope;
 import org.apache.helix.util.StringTemplate;
 
 
 /**
  *  Defines the various scopes of Helix locks, and how they are represented on Zookeeper
  */
-public class HelixLockScope {
+public class HelixLockScope implements LockScope {
 
   /**
    * Define various properties of Helix lock, and associate them with the number of arguments required for getting znode path
@@ -43,30 +44,30 @@ public class HelixLockScope {
     PARTITION(4);
 
     //the number of arguments required to generate a full path for the specific scope
-    final int _zkPathArgNum;
+    final int _pathArgNum;
 
     /**
      * Initialize a LockScopeProperty
-     * @param zkPathArgNum the number of arguments required to generate a full path for the specific scope
-\     */
-    private LockScopeProperty(int zkPathArgNum) {
-      _zkPathArgNum = zkPathArgNum;
+     * @param pathArgNum the number of arguments required to generate a full path for the specific scope
+    \     */
+    private LockScopeProperty(int pathArgNum) {
+      _pathArgNum = pathArgNum;
     }
 
     /**
      * Get the number of template arguments required to generate a full path
      * @return number of template arguments in the path
      */
-    public int getZkPathArgNum() {
-      return _zkPathArgNum;
+    public int getPathArgNum() {
+      return _pathArgNum;
     }
 
     /**
      * Get the position of this argument from the input that used to generate the scope
      * @return the number of position of value for this property in the list of keys input
      */
-    public int getArgumentPos() {
-      return _zkPathArgNum - 1;
+    public int getArgPos() {
+      return _pathArgNum - 1;
     }
   }
 
@@ -98,33 +99,33 @@ public class HelixLockScope {
    * @param type the scope
    * @param zkPathKeys keys identifying a ZNode location
    */
-  public HelixLockScope(HelixLockScope.LockScopeProperty type, List<String> zkPathKeys) {
+  private HelixLockScope(HelixLockScope.LockScopeProperty type, List<String> zkPathKeys) {
 
-    if (zkPathKeys.size() != type.getZkPathArgNum()) {
+    if (zkPathKeys.size() != type.getPathArgNum()) {
       throw new IllegalArgumentException(
-          type + " requires " + type.getZkPathArgNum() + " arguments to get znode, but was: "
+          type + " requires " + type.getPathArgNum() + " arguments to get znode, but was: "
               + zkPathKeys);
     }
 
     _type = type;
 
     //Initialize the name fields for various scope
-    _clusterName = zkPathKeys.get(LockScopeProperty.CLUSTER.getArgumentPos());
+    _clusterName = zkPathKeys.get(LockScopeProperty.CLUSTER.getArgPos());
 
-    if (type.getZkPathArgNum() >= LockScopeProperty.PARTICIPANT.getZkPathArgNum()) {
-      _participantName = zkPathKeys.get(LockScopeProperty.PARTICIPANT.getArgumentPos());
+    if (type.getPathArgNum() >= LockScopeProperty.PARTICIPANT.getPathArgNum()) {
+      _participantName = zkPathKeys.get(LockScopeProperty.PARTICIPANT.getArgPos());
     } else {
       _participantName = null;
     }
 
-    if (type.getZkPathArgNum() >= LockScopeProperty.RESOURCE.getZkPathArgNum()) {
-      _resourceName = zkPathKeys.get(LockScopeProperty.RESOURCE.getArgumentPos());
+    if (type.getPathArgNum() >= LockScopeProperty.RESOURCE.getPathArgNum()) {
+      _resourceName = zkPathKeys.get(LockScopeProperty.RESOURCE.getArgPos());
     } else {
       _resourceName = null;
     }
 
-    if (type.getZkPathArgNum() >= LockScopeProperty.PARTITION.getZkPathArgNum()) {
-      _partitionName = zkPathKeys.get(LockScopeProperty.PARTITION.getArgumentPos());
+    if (type.getPathArgNum() >= LockScopeProperty.PARTITION.getPathArgNum()) {
+      _partitionName = zkPathKeys.get(LockScopeProperty.PARTITION.getArgPos());
     } else {
       _partitionName = null;
     }
@@ -172,11 +173,11 @@ public class HelixLockScope {
     return _partitionName;
   }
 
+  @Override
   /**
    * Get the path to the corresponding ZNode
    * @return a Zookeeper path
-   */
-  public String getZkPath() {
+   */ public String getPath() {
     return _zkPath;
   }
 }
diff --git a/helix-lock/src/main/java/org/apache/helix/lock/ZKHelixNonblockingLock.java b/helix-lock/src/main/java/org/apache/helix/lock/helix/ZKHelixNonblockingLock.java
similarity index 91%
rename from helix-lock/src/main/java/org/apache/helix/lock/ZKHelixNonblockingLock.java
rename to helix-lock/src/main/java/org/apache/helix/lock/helix/ZKHelixNonblockingLock.java
index b0cc8f1..478be95 100644
--- a/helix-lock/src/main/java/org/apache/helix/lock/ZKHelixNonblockingLock.java
+++ b/helix-lock/src/main/java/org/apache/helix/lock/helix/ZKHelixNonblockingLock.java
@@ -17,23 +17,21 @@
  * under the License.
  */
 
-package org.apache.helix.lock;
+package org.apache.helix.lock.helix;
 
 import java.util.Date;
-import java.util.concurrent.locks.Lock;
 
 import org.I0Itec.zkclient.DataUpdater;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.lock.HelixLock;
+import org.apache.helix.lock.LockInfo;
+import org.apache.helix.lock.LockScope;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.log4j.Logger;
 
-import static org.apache.helix.lock.LockInfo.DEFAULT_OWNER_TEXT;
-import static org.apache.helix.lock.LockInfo.DEFAULT_TIMEOUT_LONG;
-import static org.apache.helix.lock.LockInfo.defaultLockInfo;
-
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper
@@ -56,9 +54,9 @@ public class ZKHelixNonblockingLock implements HelixLock {
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKHelixNonblockingLock(HelixLockScope scope, String zkAddress, Long timeout,
-      String lockMsg, String userId) {
-    this(scope.getZkPath(), zkAddress, timeout, lockMsg, userId);
+  public ZKHelixNonblockingLock(LockScope scope, String zkAddress, Long timeout, String lockMsg,
+      String userId) {
+    this(scope.getPath(), zkAddress, timeout, lockMsg, userId);
   }
 
   /**
@@ -154,8 +152,7 @@ public class ZKHelixNonblockingLock implements HelixLock {
    * @return true if the lock has a current owner that the ownership has not be timed out, otherwise false
    */
   private boolean hasNonExpiredOwner(ZNRecord znRecord) {
-    String owner = LockInfo.getOwner(znRecord);
-    return !owner.equals(DEFAULT_OWNER_TEXT) && !hasTimedOut(znRecord);
+    return LockInfo.ownerIdSet(znRecord) && !hasTimedOut(znRecord);
   }
 
   /**
diff --git a/helix-lock/src/test/java/org/apache/helix/lock/TestZKHelixNonblockingLock.java b/helix-lock/src/test/java/org/apache/helix/lock/helix/TestZKHelixNonblockingLock.java
similarity index 97%
rename from helix-lock/src/test/java/org/apache/helix/lock/TestZKHelixNonblockingLock.java
rename to helix-lock/src/test/java/org/apache/helix/lock/helix/TestZKHelixNonblockingLock.java
index 5e28688..e79922d 100644
--- a/helix-lock/src/test/java/org/apache/helix/lock/TestZKHelixNonblockingLock.java
+++ b/helix-lock/src/test/java/org/apache/helix/lock/helix/TestZKHelixNonblockingLock.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.helix.lock;
+package org.apache.helix.lock.helix;
 
 import java.util.ArrayList;
 import java.util.Date;
@@ -29,6 +29,7 @@ import java.util.concurrent.Callable;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.common.ZkTestBase;
+import org.apache.helix.lock.LockInfo;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
@@ -61,7 +62,7 @@ public class TestZKHelixNonblockingLock extends ZkTestBase {
     pathKeys.add("partition_name");
 
     _participantScope = new HelixLockScope(HelixLockScope.LockScopeProperty.PARTITION, pathKeys);
-    _lockPath = _participantScope.getZkPath();
+    _lockPath = _participantScope.getPath();
     _lock = new ZKHelixNonblockingLock(_participantScope, ZK_ADDR, Long.MAX_VALUE,
         _lockMessage, _userId);
   }