You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by GitBox <gi...@apache.org> on 2020/11/30 17:22:17 UTC

[GitHub] [helix] zhangmeng916 opened a new pull request #1564: Implement Helix lock priority and notification

zhangmeng916 opened a new pull request #1564:
URL: https://github.com/apache/helix/pull/1564


   ### Issues
   
   - [X] My PR addresses the following Helix issues and references them in the PR description:
   
   Fixed #1563 
   
   ### Description
   
   - [X] Here are some details about my PR, including screenshots of any UI changes:
   
   Current version of Helix lock does not support priority or preemption, meaning that if a client acquires a lock, no other client can grab it until it's timed out or released. This restricts the usage of Helix lock. In many occasions, priority and preemption is needed for lock, and preferably, when the lower priority client is preempted, it could be notified and given some time to bring the system back to stable state, e.g. do some rollback, etc. 
   In this PR, we define a few concepts of the lock, like priority, different timeout, and whether the lock is forceful. These parameters of a lock will determine how Helix would handle the acquire request.
   We also leverage Zookeeper to perform the notification and client would need to implement the cleanup callback if it is preempted.
   
   ### Tests
   
   - [X] The following tests are written for this issue:
   TestZKHelixNonblockingLockWithPriority
   
   - [ ] The following is the result of the "mvn test" command on the appropriate module:
   
   (Before CI test pass, please copy & paste the result of "mvn test")
   
   ### Documentation (Optional)
   
   - In case of new functionality, my PR adds documentation in the following wiki page:
   
   (Link the GitHub wiki you added)
   
   ### Commits
   
   - My commits all reference appropriate Apache Helix GitHub issues in their subject lines. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Code Quality
   
   - My diff has been formatted using helix-style.xml 
   (helix-style-intellij.xml if IntelliJ IDE is used)
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r538941747



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       I understand logic-wise, the API will always work. But the question is that API is not easy for the users to use. It requires much knowledge and there are invisible dependencies here and there.
   A clean API should be either one of the following cases,
   1. No matter how the parameters look like, ALL of them works in an intuitive way.
   2. If the parameters are not valid, then throw an exception to reject the call.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r538943958



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _isPending) {
+      onDeniedPendingLockNotification();
+    }
+  }
+
+  /**
+   * call back called when the lock is acquired
+   */
+  public void onAcquiredLockNotification() {
+    synchronized (ZKDistributedNonblockingLock.this) {
+      _isLocked = true;

Review comment:
       But anyway this should be resolved when you change to use enum instead of booleans, right?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535843112



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
+
+      // higher priority lock request will preempt current lock owner that is with lower priority
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority() && curLockInfo
+          .getRequestorId().equals("NONE")) {
+        // update lock Znode with requestor information
+        ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+        _isPending = true;

Review comment:
       There is no notification that would be received by the requestor before it gets the lock. So we cannot se the pending state for it. The notification it receives is that the previous owner finished the cleanup work and it acquires the lock. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r534397341



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -19,24 +19,24 @@
  * under the License.
  */
 
+import org.apache.helix.lock.helix.LockConstants;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 
 
 /**
  * Structure represents a lock node information, implemented using ZNRecord
  */
 public class LockInfo {
-
-  // Default values for each attribute if there are no current values set by user
-  public static final String DEFAULT_OWNER_TEXT = "";
-  public static final String DEFAULT_MESSAGE_TEXT = "";
-  public static final long DEFAULT_TIMEOUT_LONG = -1L;
-
   // default lock info represents the status of a unlocked lock
   public static final LockInfo defaultLockInfo =
-      new LockInfo(DEFAULT_OWNER_TEXT, DEFAULT_MESSAGE_TEXT, DEFAULT_TIMEOUT_LONG);
-
-  private static final String ZNODE_ID = "LOCK";
+      new LockInfo(LockConstants.DEFAULT_OWNER_TEXT, LockConstants.DEFAULT_MESSAGE_TEXT,

Review comment:
       Should be new LockInfo(); ?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord

Review comment:
       nit, since you will go through the whole list anyway here, the "this()" call is not necessary.
   The current code will set all the fields twice, which is not necessary at all.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       For the lock requestor that is going to enforce the lock, there is no requestor information. So what should they put for the requestor fields?
   I suggest either we change to builder pattern, or we make those parameters optional.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/LockConstants.java
##########
@@ -0,0 +1,37 @@
+package org.apache.helix.lock.helix;
+
+/*
+ * 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.
+ */
+
+
+/*
+ * Default values for each attribute if there are no current values set by user
+ */
+public class LockConstants {
+  public static final String DEFAULT_OWNER_TEXT = "";
+  public static final String DEFAULT_MESSAGE_TEXT = "";
+  public static final long DEFAULT_TIMEOUT_LONG = -1;
+  public static final int DEFAULT_PRIORITY_INT = -1;
+  public static final long DEFAULT_WAITING_TIMEOUT_LONG = -1;
+  public static final long DEFAULT_CLEANUP_TIMEOUT_LONG = -1;
+  public static final String DEFAULT_REQUESTOR_ID = "";

Review comment:
       Empty strings are not self-explainable. I suggest using something like "NONE", or "NA", or "UNKNOWN"

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(), "");
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(), "");
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(), -1);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(), -1);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(), -1);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(), -1);

Review comment:
       Why not refer to the DEFAULT lock values in the constant class?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,
+      long cleanupTimeout, String requestorId, int requestorPriority, long requestorWaitingTimeout,
+      long requestorRequestingTimestamp) {
     this();

Review comment:
       this() means set all fields 2 times.
   It could be as simple as "_record = new ZNRecord(ZNODE_ID);"

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,155 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
     }
-    _timeout = timeout;
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
+    }
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);

Review comment:
       I didn't see when this subscription is removed. Did I miss anything?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {

Review comment:
       The reason to put wait in the while loop is to prevent if anyone else updates the content after the notification and before the current thread regains the lock. I think what Junkai said is a valid comment.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,155 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
     }
-    _timeout = timeout;
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
+    }
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,

Review comment:
       Not related to this PR directly, but the isCurrentOwner() call inside the LockUpdater.update() triggers one more ZK call which is not necessary since you already have the current ZNRecord input. Please refine that logic.
   The logic is not only suboptimal but also buggy since the update logic is not purely relying on the input current znode. This potentially fails when some race condition happens.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,155 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
     }
-    _timeout = timeout;
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
+    }
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);

Review comment:
       If update() fails, it throws Exception, right? Shall we try-catch it and return false instead?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,155 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {

Review comment:
       If it is an empty callback, why not just pass null? So the lock won't trigger the callback at all.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r533672387



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);

Review comment:
       This wait() will be signaled by the following notify(), or if it never receives the notify, it'll wait until timeout.
   
   public void onAcquiredLockNotification() {
       synchronized (ZKDistributedNonblockingLock.this) {
         _isLocked = true;
         ZKDistributedNonblockingLock.this.notify();
       }
     }




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
alirezazamani commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535433085



##########
File path: helix-lock/src/test/java/org/apache/helix/lock/helix/TestZKHelixNonblockingLockWithPriority.java
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.helix;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.helix.HelixException;
+import org.apache.helix.TestHelper;
+import org.apache.helix.common.ZkTestBase;
+import org.testng.Assert;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+
+public class TestZKHelixNonblockingLockWithPriority extends ZkTestBase {
+
+  private final String _clusterName = TestHelper.getTestClassName();
+  private String _lockPath;
+  private HelixLockScope _participantScope;
+  private AtomicBoolean _isCleanupNotified;
+
+  private final LockListener _lockListener = new LockListener() {
+    @Override
+    public void onCleanupNotification() {
+      _isCleanupNotified.set(true);
+      try {
+        Thread.sleep(20000);

Review comment:
       Ok. Makes sense




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r534599345



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {

Review comment:
       If there's another user (whose priority is even higher) acquired the lock, the onDeniedPendingLockNotification will be called, which will terminate the wait. We won't need to worry about this, right?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,155 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
     }
-    _timeout = timeout;
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
+    }
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);

Review comment:
       If you look at doUpdate in ZkbaseDataAccessor, it actually never throws an exception. All exceptions are caught and a boolean value is returned. 

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -19,24 +19,24 @@
  * under the License.
  */
 
+import org.apache.helix.lock.helix.LockConstants;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 
 
 /**
  * Structure represents a lock node information, implemented using ZNRecord
  */
 public class LockInfo {
-
-  // Default values for each attribute if there are no current values set by user
-  public static final String DEFAULT_OWNER_TEXT = "";
-  public static final String DEFAULT_MESSAGE_TEXT = "";
-  public static final long DEFAULT_TIMEOUT_LONG = -1L;
-
   // default lock info represents the status of a unlocked lock
   public static final LockInfo defaultLockInfo =
-      new LockInfo(DEFAULT_OWNER_TEXT, DEFAULT_MESSAGE_TEXT, DEFAULT_TIMEOUT_LONG);
-
-  private static final String ZNODE_ID = "LOCK";
+      new LockInfo(LockConstants.DEFAULT_OWNER_TEXT, LockConstants.DEFAULT_MESSAGE_TEXT,

Review comment:
       Yeah, you're right. I was following previous pattern.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord

Review comment:
       This one is actually a bit tricky, the znRecord could be null when there is no lock, and we need to call the private constructor to set the default value. So this() has to be there. 

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       When issuing an acquire lock request, client always put their own ID in owner field, and the requestor field is null. If later Helix finds there is already an owner, it'll put the new request into requestor field in case the priority is higher. This is true no matter whether it is a forceful lock or not. Forceful only means if the cleanup work of lower priority lock is not finished during timeout, whether the higher priority forcefully grab the lock.

##########
File path: helix-lock/src/test/java/org/apache/helix/lock/helix/TestZKHelixNonblockingLockWithPriority.java
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.helix;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.helix.HelixException;
+import org.apache.helix.TestHelper;
+import org.apache.helix.common.ZkTestBase;
+import org.testng.Assert;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+
+public class TestZKHelixNonblockingLockWithPriority extends ZkTestBase {
+
+  private final String _clusterName = TestHelper.getTestClassName();
+  private String _lockPath;
+  private HelixLockScope _participantScope;
+  private AtomicBoolean _isCleanupNotified;
+
+  private final LockListener _lockListener = new LockListener() {
+    @Override
+    public void onCleanupNotification() {
+      _isCleanupNotified.set(true);
+      try {
+        Thread.sleep(20000);

Review comment:
       This sleep is a real sleep. I would like the thread to run for 20 seconds, and then exit. So this should not cause the problem we saw before, right?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,155 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
     }
-    _timeout = timeout;
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
+    }
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);

Review comment:
       Updated with unsubscription.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data

Review comment:
       The reason I was doing another read is that I wasn't using lock updater, but use a normal updater. I feel for these callback cases, we should not use lock updater, which is used during first tryLock. The reason is that we usually know what we are doing in the callback updater, and put all these logic inside lock updater will make that logic very complicated and hard to read. 
   For example, in the lock updater, we always assume there is a normal lock request, which has owner ID, etc. However, the cleanup callback updater actually removes the owner ID. Do you think we should combine all scenarios in one updater?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();

Review comment:
       We don't know when the customers would start to do the cleanup. Do you mean we require them to update Znode during their cleanup callback? I don't think it's recommended to let them directly update Znode, right?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(), "");
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(), "");
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(), -1);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(), -1);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(), -1);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(), -1);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {

Review comment:
       The if condition is:
   if (lockInfo.getOwner().equals(_userId))
   
   Basically we need to distinguish between requestor and owner. Both of them may receive callback, but they don't need to handle unrelated callback. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539633214



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
+
+      // higher priority lock request will preempt current lock owner that is with lower priority
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority() && curLockInfo
+          .getRequestorId().equals("NONE")) {
+        // update lock Znode with requestor information
+        ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+        _isPending = true;

Review comment:
       I see. With the enum status instead of booleans, the logic has been clean enough. So I think it is good enough.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539035263



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       I cannot find the validation method though.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539004374



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
+
+      // higher priority lock request will preempt current lock owner that is with lower priority
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority() && curLockInfo
+          .getRequestorId().equals("NONE")) {
+        // update lock Znode with requestor information
+        ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+        _isPending = true;

Review comment:
       Notification would be received by all listeners. But for requestor, it won't handle its own preempting request. Only owner needs to start cleanup. Requestor only handle data change when owner finishes the cleanup work and remove itself from the owner field. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r533672874



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +218,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {

Review comment:
       Yeah, we are using this one to trigger a onAcquiredLockNotification callback, which will notify the waiting thread.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539035836



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/LockConstants.java
##########
@@ -0,0 +1,40 @@
+package org.apache.helix.lock.helix;
+
+/*
+ * 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.
+ */
+
+
+/*
+ * Default values for each attribute if there are no current values set by user
+ */
+public class LockConstants {
+  public static final String DEFAULT_USER_ID = "NONE";
+  public static final String DEFAULT_MESSAGE_TEXT = "NONE";
+  public static final long DEFAULT_TIMEOUT_LONG = -1;
+  public static final int DEFAULT_PRIORITY_INT = -1;
+  public static final long DEFAULT_WAITING_TIMEOUT_LONG = -1;
+  public static final long DEFAULT_CLEANUP_TIMEOUT_LONG = -1;
+  public static final long DEFAULT_REQUESTING_TIMESTAMP_LONG = -1;
+
+  public enum LockStatus {

Review comment:
       nit, please comment on the meaning of these statuses, especially PREEMPTED : )




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] pkuwm commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535859104



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,

Review comment:
       It seems `leaseTimeout` does not need `Long` boxing. `long` is fine?
   
   And, by seeing the list of params, just wow... So many params :)
   Can we encapsulate some of the params into an object, eg. `ZkDistributedLockProperties` or `ZkDistributedLockConfig`.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535843230



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _isPending) {
+      onDeniedPendingLockNotification();
+    }
+  }
+
+  /**
+   * call back called when the lock is acquired
+   */
+  public void onAcquiredLockNotification() {
+    synchronized (ZKDistributedNonblockingLock.this) {
+      _isLocked = true;

Review comment:
       Please see the comments below. Here the status should be LOCKED.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r533658562



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -30,13 +30,23 @@
   // Default values for each attribute if there are no current values set by user
   public static final String DEFAULT_OWNER_TEXT = "";
   public static final String DEFAULT_MESSAGE_TEXT = "";
-  public static final long DEFAULT_TIMEOUT_LONG = -1L;
+  public static final long DEFAULT_TIMEOUT_LONG = -1;
+  public static final int DEFAULT_PRIORITY_INT = -1;
+  public static final long DEFAULT_WAITING_TIMEOUT_LONG = -1;
+  public static final long DEFAULT_CLEANUP_TIMEOUT_LONG = -1;
+  public static final String DEFAULT_REQUESTOR_ID = "";
+  public static final int DEFAULT_REQUESTOR_PRIORITY_INT = -1;
+  public static final long DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG = -1;
+  public static final long DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG = -1;

Review comment:
       Shall we have a LockConstant class to hold them? Seemed they are too many in same class with logic code.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);

Review comment:
       Are we just waiting until timeout and do a check? Shall we periodically wake up and check whether the lock has been released or not?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {

Review comment:
       Usually, if we are waiting for something, we should use while to do the checks here.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +218,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {

Review comment:
       Can we leverage this handleDataChange to notify the waiting thread? Then we can use some Java internal lock.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539480027



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -32,68 +33,167 @@
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private LockConstants.LockStatus _lockStatus;
+  private long _pendingTimeout;
+  private CountDownLatch _countDownLatch = new CountDownLatch(1);
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,

Review comment:
       I believe it does support multi zk. Underlying it calls 
   _zkClient = buildRealmAwareZkClientWithDefaultConfigs(
           new RealmAwareZkClient.RealmAwareZkClientConfig().setZkSerializer(zkSerializer), zkAddress,
           zkClientType);
   If the zk address is null, it'll use default system property config to build a multi zk client. Right?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani merged pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
alirezazamani merged pull request #1564:
URL: https://github.com/apache/helix/pull/1564


   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539003727



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _isPending) {
+      onDeniedPendingLockNotification();
+    }
+  }
+
+  /**
+   * call back called when the lock is acquired
+   */
+  public void onAcquiredLockNotification() {
+    synchronized (ZKDistributedNonblockingLock.this) {
+      _isLocked = true;

Review comment:
       Yeah, it's already changed to an enum. Please see the updated code.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539035263



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       I didn't see the validation method though.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       I cannot find the validation method though.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r540613949



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -32,68 +33,167 @@
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private LockConstants.LockStatus _lockStatus;
+  private long _pendingTimeout;
+  private CountDownLatch _countDownLatch = new CountDownLatch(1);
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false, null,
+        new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
-   * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
+   * Initialize the lock with ZKLockConfig. This is the preferred way to construct the lock.
+   */
+  public ZKDistributedNonblockingLock(ZKLockConfig zkLockConfig) {
+    this(zkLockConfig.getLockScope(), zkLockConfig.getZkAddress(), zkLockConfig.getLeaseTimeout(),
+        zkLockConfig.getLockMsg(), zkLockConfig.getUserId(), zkLockConfig.getPriority(),
+        zkLockConfig.getWaitingTimeout(), zkLockConfig.getCleanupTimeout(),
+        zkLockConfig.getIsForceful(), zkLockConfig.getLockListener());
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, priority,
+   * different kinds of timeout, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  private ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Internal construction of the lock with user provided information, e.g., lock path under
+   * zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
+    validateInput();
   }
 
   @Override
   public boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Immediately return if the lock statue is not PENDING.
+    if (_lockStatus != LockConstants.LockStatus.PENDING) {

Review comment:
       If I understand correctly, is the concern regarding the race condition between here and the following part?
     public void onAcquiredLockNotification() {
       _lockStatus = LockConstants.LockStatus.LOCKED;
       _countDownLatch.countDown();
     }
   
   It seems fine. Even the Lock status change and count down happen before await, the await will still exist when timeout, and the lock status is retrieved again after await.
   I tried to leverage synchronized on both parts_lockstatus, but it doesn't work, cause it'll deadlock. Let me know if you still think it's not safe. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r537980108



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {

Review comment:
       Yeah, it should work. This is the design in MVP, and I think it has been tested by customers. 
   The logic is, if there is no znode existing, we create LockInfo with default value, and take that as current value, update that value with user's requested lock information.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on pull request #1564:
URL: https://github.com/apache/helix/pull/1564#issuecomment-744729607


   This PR is ready to be merged. Approved by @jiajunwang 
   Final commit message:
   Implement priority for Helix distributed lock.
   - Add priority definition and different kinds of timeout definition for Helix lock.
   - Add notification support during high priority lock request preempting low priority lock.
   - Add forceful lock concept when the cleanup work of low priority lock is not finished in the timeout. 


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r538940088



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord

Review comment:
       It would still be cleaner to just do a simple check instead of initializing the values 2 times.
   For instance,
   if (znRecord == null) {
     znRecord = new znRecord();
   }
   ...
   Then the default value will for sure be set in the same way since this record is empty.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] alirezazamani commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
alirezazamani commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r534413477



##########
File path: helix-lock/src/test/java/org/apache/helix/lock/helix/TestZKHelixNonblockingLockWithPriority.java
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.helix;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.helix.HelixException;
+import org.apache.helix.TestHelper;
+import org.apache.helix.common.ZkTestBase;
+import org.testng.Assert;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+
+public class TestZKHelixNonblockingLockWithPriority extends ZkTestBase {
+
+  private final String _clusterName = TestHelper.getTestClassName();
+  private String _lockPath;
+  private HelixLockScope _participantScope;
+  private AtomicBoolean _isCleanupNotified;
+
+  private final LockListener _lockListener = new LockListener() {
+    @Override
+    public void onCleanupNotification() {
+      _isCleanupNotified.set(true);
+      try {
+        Thread.sleep(20000);

Review comment:
       Can this sleep be avoided?

##########
File path: helix-lock/src/test/java/org/apache/helix/lock/helix/TestZKHelixNonblockingLockWithPriority.java
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.helix;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.helix.HelixException;
+import org.apache.helix.TestHelper;
+import org.apache.helix.common.ZkTestBase;
+import org.testng.Assert;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+
+public class TestZKHelixNonblockingLockWithPriority extends ZkTestBase {
+
+  private final String _clusterName = TestHelper.getTestClassName();
+  private String _lockPath;
+  private HelixLockScope _participantScope;
+  private AtomicBoolean _isCleanupNotified;
+
+  private final LockListener _lockListener = new LockListener() {
+    @Override
+    public void onCleanupNotification() {
+      _isCleanupNotified.set(true);
+      try {
+        Thread.sleep(20000);
+      } catch (Exception e) {
+
+      }
+    }
+  };
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    System.out.println("START " + _clusterName + " at " + new Date(System.currentTimeMillis()));
+
+    TestHelper.setupCluster(_clusterName, ZK_ADDR, 12918, "localhost", "TestDB", 1, 10, 5, 3,
+        "MasterSlave", true);
+
+    List<String> pathKeys = new ArrayList<>();
+    pathKeys.add(_clusterName);
+    pathKeys.add(_clusterName);
+    _participantScope = new HelixLockScope(HelixLockScope.LockScopeProperty.CLUSTER, pathKeys);
+    _lockPath = _participantScope.getPath();
+
+    _isCleanupNotified = new AtomicBoolean(false);
+  }
+
+  @BeforeMethod
+  public void beforeMethod() {
+    _gZkClient.delete(_lockPath);
+    Assert.assertFalse(_gZkClient.exists(_lockPath));
+  }
+
+  @AfterSuite
+  public void afterSuite() throws IOException {
+    super.afterSuite();
+  }
+
+  @Test
+  public void testLowerPriorityRequestRejected() throws Exception {
+    ZKDistributedNonblockingLock lock =
+        new ZKDistributedNonblockingLock(_participantScope, ZK_ADDR, 3600000L, "original "
+            + "lock", "original_lock", 1, 1000, 25000, false, _lockListener);
+
+    ZKDistributedNonblockingLock lowerLock = new ZKDistributedNonblockingLock(_participantScope,
+        ZK_ADDR, 3600000L,
+        "lower priority lock", "low_lock", 0, 1000, 2000, false, createLockListener());
+
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        lock.tryLock();
+      }
+    };
+
+    t.start();
+    t.join();
+    Assert.assertTrue(lock.isCurrentOwner());
+
+    LockHelper lockHelper = new LockHelper(lowerLock);
+    Thread threadLow = new Thread(lockHelper);
+    threadLow.start();
+    threadLow.join();
+    boolean lowResult = lockHelper.getResult();
+
+    Assert.assertFalse(_isCleanupNotified.get());
+    Assert.assertFalse(lowerLock.isCurrentOwner());
+    Assert.assertFalse(lowResult);
+    lock.unlock();
+    lock.close();
+    lowerLock.close();
+  }
+
+  @Test
+  public void testHigherPriorityRequestAcquired() throws Exception {
+    ZKDistributedNonblockingLock lock =
+        new ZKDistributedNonblockingLock(_participantScope, ZK_ADDR, 3600000L, "original "
+            + "lock", "original_lock", 1, 1000, 25000, false, _lockListener);
+
+    // The waitingTimeout of higher priority request is larger than cleanup time of current owner
+    ZKDistributedNonblockingLock higherLock = new ZKDistributedNonblockingLock(_participantScope,
+        ZK_ADDR, 3600000L,
+        "higher priority lock", "high_lock", 2, 30000, 10000, false, createLockListener());
+
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        lock.tryLock();
+      }
+    };
+
+    t.start();
+    t.join();
+    Assert.assertTrue(lock.isCurrentOwner());
+
+    LockHelper lockHelper = new LockHelper(higherLock);
+    Thread t_higher = new Thread(lockHelper);
+    t_higher.start();
+    t_higher.join();
+    boolean higherResult = lockHelper.getResult();
+
+    Assert.assertTrue(_isCleanupNotified.get());
+    Assert.assertTrue(higherLock.isCurrentOwner());
+    Assert.assertTrue(higherResult);
+    _isCleanupNotified.set(false);
+    higherLock.unlock();
+    higherLock.close();
+    lock.close();
+  }
+
+  @Test
+  public void testHigherPriorityRequestFailedAsCleanupHasNotDone() throws Exception {
+    ZKDistributedNonblockingLock lock =
+        new ZKDistributedNonblockingLock(_participantScope, ZK_ADDR, 3600000L, "original "
+            + "lock", "original_lock", 1, 1000, 25000, false, _lockListener);
+
+    // The waitingTimeout of higher priority request is shorter than cleanup time of current
+    // owner, and the higher priority request is not forceful.
+    ZKDistributedNonblockingLock higherLock_short =
+        new ZKDistributedNonblockingLock(_participantScope, ZK_ADDR, 3600000L,
+            "higher priority lock short", "high_lock_short", 2, 2000, 10000, false, createLockListener());
+
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        lock.tryLock();
+      }
+    };
+
+    t.start();
+    t.join();
+    Assert.assertTrue(lock.isCurrentOwner());
+
+    LockHelper lockHelper = new LockHelper(higherLock_short);
+    Thread t_higher = new Thread(lockHelper);
+
+    t_higher.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+      @Override
+      public void uncaughtException(Thread th, Throwable ex) {
+        Assert.assertTrue(ex.getMessage().contains("Clean up has not finished by lock owner"));
+      }
+    });
+    t_higher.start();
+    t_higher.join();
+    boolean higherResult = lockHelper.getResult();
+
+    Assert.assertTrue(_isCleanupNotified.get());
+    Assert.assertFalse(higherLock_short.isCurrentOwner());
+    Assert.assertFalse(higherResult);
+    _isCleanupNotified.set(false);
+
+    Assert.assertTrue(lock.unlock());
+    Assert.assertFalse(lock.isCurrentOwner());
+    lock.close();
+    higherLock_short.close();
+  }
+
+  @Test
+  public void testHigherPriorityRequestForcefulAcquired() throws Exception {
+    ZKDistributedNonblockingLock lock =
+        new ZKDistributedNonblockingLock(_participantScope, ZK_ADDR, 3600000L, "original "
+            + "lock", "original_lock", 1, 1000, 25000, false, _lockListener);
+
+    // The waitingTimeout of higher priority request is shorter than cleanup time of current
+    // owner, but the higher priority request is forceful.
+    ZKDistributedNonblockingLock higherLock_force =
+        new ZKDistributedNonblockingLock(_participantScope, ZK_ADDR, 3600000L,
+            "higher priority lock force", "high_lock_force", 2, 2000, 10000, true,
+            new LockListener() {
+              @Override
+              public void onCleanupNotification() {
+              }
+            });
+
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        lock.tryLock();
+      }
+    };
+
+    t.start();
+    t.join();
+    Assert.assertTrue(lock.isCurrentOwner());
+
+    LockHelper lockHelper = new LockHelper(higherLock_force);
+    Thread t_higher = new Thread(lockHelper);
+
+    t_higher.start();
+    t_higher.join();
+    boolean higherResult = lockHelper.getResult();
+
+    boolean res = TestHelper.verify(() -> {
+      return _isCleanupNotified.get();
+    }, TestHelper.WAIT_DURATION);
+    Assert.assertTrue(res);
+    Assert.assertTrue(higherLock_force.isCurrentOwner());
+    Assert.assertTrue(higherResult);
+    _isCleanupNotified.set(false);
+
+    higherLock_force.unlock();
+    higherLock_force.close();
+    lock.close();
+  }
+
+  @Test
+  public void testHigherPriorityRequestPreemptedByAnother() throws Exception {
+    ZKDistributedNonblockingLock lock =
+        new ZKDistributedNonblockingLock(_participantScope, ZK_ADDR, 3600000L, "original "
+            + "lock", "original_lock", 1, 1000, 25000, false, _lockListener);
+
+    ZKDistributedNonblockingLock higherLock = new ZKDistributedNonblockingLock(_participantScope,
+        ZK_ADDR, 3600000L,
+        "higher priority lock", "high_lock", 2, 30000, 10000, false, createLockListener());
+
+    ZKDistributedNonblockingLock highestLock = new ZKDistributedNonblockingLock(_participantScope
+        , ZK_ADDR, 3600000L,
+        "highest priority lock", "highest_lock", 3, 30000, 20000, false, createLockListener());
+
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        lock.tryLock();
+      }
+    };
+
+    Thread t_highest = new Thread() {
+      @Override
+      public void run() {
+        highestLock.tryLock();
+      }
+    };
+
+    t.start();
+    t.join();
+    Assert.assertTrue(lock.isCurrentOwner());
+
+    LockHelper lockHelper = new LockHelper(higherLock);
+    Thread t_higher = new Thread(lockHelper);
+    t_higher.start();
+    Thread.sleep(1000);

Review comment:
       Can we avoid the sleep here? Maybe it is better to use TestHelper methods.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r534421423



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo

Review comment:
       The Default ID is "" in this version, right? I guess we should check == null || .isEmpty() ?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");

Review comment:
       Verbose log. debug?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();

Review comment:
       Not about code, but this means we don't have control on when the cleanup starts.
   Would it be possible that the cleaner lock holder also updates the node so we are sure that cleaning up has been started?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +303,88 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner()) {

Review comment:
       Same here, I'm concerned about the isCurrentOwner usage inside the update method.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(), "");
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(), "");
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(), -1);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(), -1);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(), -1);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(), -1);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {

Review comment:
       I cannot match it easily with the if condition above, but they look like the same : )

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data

Review comment:
       Could you please elaborate on this part? I didn't get the necessity of reading again.
   You can relies on the updater to check the ownership for you, right? If the owner has been changed in between, then the update fails and that is OK.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539640437



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -32,68 +33,167 @@
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private LockConstants.LockStatus _lockStatus;
+  private long _pendingTimeout;
+  private CountDownLatch _countDownLatch = new CountDownLatch(1);
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false, null,
+        new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
-   * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
+   * Initialize the lock with ZKLockConfig. This is the preferred way to construct the lock.
+   */
+  public ZKDistributedNonblockingLock(ZKLockConfig zkLockConfig) {
+    this(zkLockConfig.getLockScope(), zkLockConfig.getZkAddress(), zkLockConfig.getLeaseTimeout(),
+        zkLockConfig.getLockMsg(), zkLockConfig.getUserId(), zkLockConfig.getPriority(),
+        zkLockConfig.getWaitingTimeout(), zkLockConfig.getCleanupTimeout(),
+        zkLockConfig.getIsForceful(), zkLockConfig.getLockListener());
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, priority,
+   * different kinds of timeout, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,

Review comment:
       I think we want users to use zooscalibility features as far as they can. So shall we hide this constructor? The above one with zkAddress is left because we want to ensure backward compatibility. But this one is newly added. So make it private?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539034573



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -45,54 +40,105 @@
   public enum LockInfoAttribute {
     OWNER,
     MESSAGE,
-    TIMEOUT
+    TIMEOUT,
+    PRIORITY,
+    WAITING_TIMEOUT,
+    CLEANUP_TIMEOUT,
+    REQUESTOR_ID,
+    REQUESTOR_PRIORITY,
+    REQUESTOR_WAITING_TIMEOUT,
+    REQUESTING_TIMESTAMP
   }
 
   /**
    * Initialize a default LockInfo instance
    */
   private LockInfo() {
     _record = new ZNRecord(ZNODE_ID);
-    setLockInfoFields(DEFAULT_OWNER_TEXT, DEFAULT_MESSAGE_TEXT, DEFAULT_TIMEOUT_LONG);
+    setLockInfoFields(LockConstants.DEFAULT_USER_ID, LockConstants.DEFAULT_MESSAGE_TEXT,
+        LockConstants.DEFAULT_TIMEOUT_LONG, LockConstants.DEFAULT_PRIORITY_INT,
+        LockConstants.DEFAULT_WAITING_TIMEOUT_LONG, LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG,
+        LockConstants.DEFAULT_USER_ID, LockConstants.DEFAULT_PRIORITY_INT,
+        LockConstants.DEFAULT_WAITING_TIMEOUT_LONG,
+        LockConstants.DEFAULT_REQUESTING_TIMESTAMP_LONG);
   }
 
   /**
    * Initialize a LockInfo with a ZNRecord, set all info fields to default data
    * @param znRecord The ZNRecord contains lock node data that used to initialize the LockInfo
    */
   public LockInfo(ZNRecord znRecord) {
-    this();
-    if (znRecord != null) {
-      String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
-      String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+    _record = new ZNRecord(ZNODE_ID);
+    if (znRecord == null) {
+      znRecord = new ZNRecord(ZNODE_ID);
     }
+    String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name()) == null
+        ? LockConstants.DEFAULT_USER_ID : znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
+    String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name()) == null
+        ? LockConstants.DEFAULT_MESSAGE_TEXT
+        : znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
+    long timeout =
+        znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+    int priority =
+        znRecord.getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long requestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,

Review comment:
       nit, combine this with line 71, then we can call the constructor directly instead of 2 lines.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r542634125



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -32,68 +33,167 @@
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private LockConstants.LockStatus _lockStatus;
+  private long _pendingTimeout;
+  private CountDownLatch _countDownLatch = new CountDownLatch(1);
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false, null,
+        new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
-   * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
+   * Initialize the lock with ZKLockConfig. This is the preferred way to construct the lock.
+   */
+  public ZKDistributedNonblockingLock(ZKLockConfig zkLockConfig) {
+    this(zkLockConfig.getLockScope(), zkLockConfig.getZkAddress(), zkLockConfig.getLeaseTimeout(),
+        zkLockConfig.getLockMsg(), zkLockConfig.getUserId(), zkLockConfig.getPriority(),
+        zkLockConfig.getWaitingTimeout(), zkLockConfig.getCleanupTimeout(),
+        zkLockConfig.getIsForceful(), zkLockConfig.getLockListener());
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, priority,
+   * different kinds of timeout, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  private ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Internal construction of the lock with user provided information, e.g., lock path under
+   * zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
+    validateInput();
   }
 
   @Override
   public boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Immediately return if the lock statue is not PENDING.
+    if (_lockStatus != LockConstants.LockStatus.PENDING) {

Review comment:
       I see your concern. By current logic, it should be safe. If the previous lock was already acquired, subsequent tryLock from the same object will directly acquire the lock. If the previous lock was pending, subsequent tryLock won't check status as they're the same priority.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539036654



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -32,68 +33,167 @@
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private LockConstants.LockStatus _lockStatus;
+  private long _pendingTimeout;
+  private CountDownLatch _countDownLatch = new CountDownLatch(1);
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,

Review comment:
       I just noticed that there is zkAddress input. So this is not multi zk supported API, right? Will that be a concern?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -32,68 +33,167 @@
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private LockConstants.LockStatus _lockStatus;
+  private long _pendingTimeout;
+  private CountDownLatch _countDownLatch = new CountDownLatch(1);
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false, null,
+        new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
-   * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
+   * Initialize the lock with ZKLockConfig. This is the preferred way to construct the lock.
+   */
+  public ZKDistributedNonblockingLock(ZKLockConfig zkLockConfig) {
+    this(zkLockConfig.getLockScope(), zkLockConfig.getZkAddress(), zkLockConfig.getLeaseTimeout(),
+        zkLockConfig.getLockMsg(), zkLockConfig.getUserId(), zkLockConfig.getPriority(),
+        zkLockConfig.getWaitingTimeout(), zkLockConfig.getCleanupTimeout(),
+        zkLockConfig.getIsForceful(), zkLockConfig.getLockListener());
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, priority,
+   * different kinds of timeout, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Internal construction of the lock with user provided information, e.g., lock path under
+   * zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
+    validateInput();
   }
 
   @Override
   public boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Immediately return if the lock statue is not PENDING.
+    if (_lockStatus != LockConstants.LockStatus.PENDING) {
+      if (!updateResult) {
+        _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
+      }
+      return updateResult;
     }
-    LockUpdater updater = new LockUpdater(new LockInfo(_userId, _lockMsg, deadline));
-    return _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // When the lock status is still pending, wait for the period recorded in _pendingTimeout.
+    try {
+      _countDownLatch.await(_pendingTimeout, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      throw new HelixException(
+          String.format("Interruption happened while %s is waiting for the " + "lock", _userId), e);

Review comment:
       nit,
   
   > the " + "lock"
   
   I assume you want to add something in between?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +305,143 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
-      LOG.error(
-          "User " + _userId + " tried to update the lock at " + new Date(System.currentTimeMillis())
-              + ". Lock path: " + _lockPath);
-      throw new HelixException("User is not authorized to perform this operation.");
+
+      // higher priority lock request will try to  preempt current lock owner
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority()) {
+        if (curLockInfo.getRequestorId().equals("NONE")) {

Review comment:
       Can we avoid hardcode NONE but refer to the default ID?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +305,143 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
-      LOG.error(
-          "User " + _userId + " tried to update the lock at " + new Date(System.currentTimeMillis())
-              + ". Lock path: " + _lockPath);
-      throw new HelixException("User is not authorized to perform this operation.");
+
+      // higher priority lock request will try to  preempt current lock owner
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority()) {
+        if (curLockInfo.getRequestorId().equals("NONE")) {
+          // update lock Znode with requestor information
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          _pendingTimeout =
+              _waitingTimeout > curLockInfo.getCleanupTimeout() ? curLockInfo.getCleanupTimeout()
+                  : _waitingTimeout;
+          return newRecord;
+        } // If the requestor field is not empty, and the coming lock request has a even higher
+        // priority. The new request will replace current requestor field of the lock
+        else if (_priority > curLockInfo.getRequestorPriority()) {
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          long remainingCleanupTime =
+              curLockInfo.getCleanupTimeout() - (System.currentTimeMillis() - curLockInfo
+                  .getRequestingTimestamp());
+          _pendingTimeout = _waitingTimeout > remainingCleanupTime ? remainingCleanupTime :
+           _waitingTimeout;
+          return newRecord;
+        }
+      }
+
+      // For users who are not the lock owner and the priority is not higher than current lock
+      // owner, or the priority is higher than current lock, but lower than the requestor, throw
+      // an exception. The exception will be caught by data accessor, and return false for the
+      // update operation.
+      LOG.error("User {} failed to acquire lock at Lock path {}.", _userId, _lockPath);
+      throw new HelixException(
+          String.format("User %s failed to acquire lock at Lock path %s.", _userId, _lockPath));
     }
   }
 
+  /**
+   * Class that specifies how a lock node should be updated after the previous owner finishes the
+   * cleanup work
+   */
+  private class CleanupUpdater implements DataUpdater<ZNRecord> {
+    public CleanupUpdater() {
+    }
+
+    @Override
+    public ZNRecord update(ZNRecord current) {
+      // If we are still the lock owner, clean owner field.
+      LockInfo curLockInfo = new LockInfo(current);
+      if (isCurrentOwner(curLockInfo)) {
+        ZNRecord record = current;
+        record
+            .setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(), LockConstants.DEFAULT_USER_ID);
+        record.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+            LockConstants.DEFAULT_MESSAGE_TEXT);
+        record.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+            LockConstants.DEFAULT_TIMEOUT_LONG);
+        record.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+            LockConstants.DEFAULT_PRIORITY_INT);
+        record.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+            LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+        record.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+            LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+        return record;
+      }
+      LOG.error("User {} is not current lock owner, and does not need to unlock {}", _userId,
+          _lockPath);
+      throw new HelixException(String
+          .format("User %s is not current lock owner, and does not need" + " to unlock %s", _userId,
+              _lockPath));
+    }
+  }
+
+  /**
+   * Class that specifies how a lock node should be updated during a forceful get lock operation
+   */
+  private class ForcefulUpdater implements DataUpdater<ZNRecord> {
+    final ZNRecord _record;
+
+    /**
+     * Initialize a structure for lock user to update a lock node value
+     * @param lockInfo the lock node value will be used to update the lock
+     */
+    public ForcefulUpdater(LockInfo lockInfo) {
+      _record = lockInfo.getRecord();
+    }
+
+    @Override
+    public ZNRecord update(ZNRecord current) {
+      // If we are still the lock requestor, change ourselves to be lock owner.
+      LockInfo curLockInfo = new LockInfo(current);
+      if (curLockInfo.getRequestorId().equals(_userId)) {
+        return _record;
+      }
+      LOG.error("User {} is not current lock requestor, and cannot forcefully acquire the lock at "
+          + "{}", _userId, _lockPath);
+      throw new HelixException(String.format("User %s is not current lock requestor, and cannot "
+          + "forcefully acquire the lock at %s", _userId, _lockPath));
+    }
+  }
+
+  private ZNRecord composeNewRequestorRecord(LockInfo oldLockInfo, ZNRecord newLockZNRecord) {

Review comment:
       nit, 
   oldLockInfo => existingLockInfo, and newLockZNRecord => requestorLockZNRecord ?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +305,143 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
-      LOG.error(
-          "User " + _userId + " tried to update the lock at " + new Date(System.currentTimeMillis())
-              + ". Lock path: " + _lockPath);
-      throw new HelixException("User is not authorized to perform this operation.");
+
+      // higher priority lock request will try to  preempt current lock owner
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority()) {
+        if (curLockInfo.getRequestorId().equals("NONE")) {
+          // update lock Znode with requestor information
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          _pendingTimeout =
+              _waitingTimeout > curLockInfo.getCleanupTimeout() ? curLockInfo.getCleanupTimeout()
+                  : _waitingTimeout;
+          return newRecord;
+        } // If the requestor field is not empty, and the coming lock request has a even higher
+        // priority. The new request will replace current requestor field of the lock
+        else if (_priority > curLockInfo.getRequestorPriority()) {
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          long remainingCleanupTime =
+              curLockInfo.getCleanupTimeout() - (System.currentTimeMillis() - curLockInfo
+                  .getRequestingTimestamp());
+          _pendingTimeout = _waitingTimeout > remainingCleanupTime ? remainingCleanupTime :
+           _waitingTimeout;
+          return newRecord;
+        }

Review comment:
       There is some duplicate code, can we do this?
   
   if (curLockInfo.getRequestorId().equals("NONE")) {
   _pendingTimeout = .....
   } else if (_priority > curLockInfo.getRequestorPriority()) {
   _pendingTimeout = ......
   }
   ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
         throw new HelixException("User is not authorized to perform this operation.");
    _lockStatus = LockConstants.LockStatus.PENDING;
   return newRecord;

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKLockConfig.java
##########
@@ -0,0 +1,173 @@
+package org.apache.helix.lock.helix;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.lock.LockScope;
+import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Hold configs used for a ZK distributed nonblocking lock.
+ */
+public class ZKLockConfig {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(org.apache.helix.lock.helix.ZKLockConfig.class.getName());
+  private LockScope _lockScope;
+  private String _zkAddress;

Review comment:
       I do have some concerns about this direct ZkAddress parameter. This won't support multi-zk usage.
   It might be OK, but the LOCK API will be treated as a special case, I guess.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +305,143 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
-      LOG.error(
-          "User " + _userId + " tried to update the lock at " + new Date(System.currentTimeMillis())
-              + ". Lock path: " + _lockPath);
-      throw new HelixException("User is not authorized to perform this operation.");
+
+      // higher priority lock request will try to  preempt current lock owner
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority()) {
+        if (curLockInfo.getRequestorId().equals("NONE")) {
+          // update lock Znode with requestor information
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          _pendingTimeout =
+              _waitingTimeout > curLockInfo.getCleanupTimeout() ? curLockInfo.getCleanupTimeout()
+                  : _waitingTimeout;
+          return newRecord;
+        } // If the requestor field is not empty, and the coming lock request has a even higher
+        // priority. The new request will replace current requestor field of the lock
+        else if (_priority > curLockInfo.getRequestorPriority()) {
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          long remainingCleanupTime =
+              curLockInfo.getCleanupTimeout() - (System.currentTimeMillis() - curLockInfo
+                  .getRequestingTimestamp());
+          _pendingTimeout = _waitingTimeout > remainingCleanupTime ? remainingCleanupTime :
+           _waitingTimeout;
+          return newRecord;
+        }
+      }
+
+      // For users who are not the lock owner and the priority is not higher than current lock
+      // owner, or the priority is higher than current lock, but lower than the requestor, throw
+      // an exception. The exception will be caught by data accessor, and return false for the
+      // update operation.
+      LOG.error("User {} failed to acquire lock at Lock path {}.", _userId, _lockPath);
+      throw new HelixException(
+          String.format("User %s failed to acquire lock at Lock path %s.", _userId, _lockPath));
     }
   }
 
+  /**
+   * Class that specifies how a lock node should be updated after the previous owner finishes the
+   * cleanup work
+   */
+  private class CleanupUpdater implements DataUpdater<ZNRecord> {
+    public CleanupUpdater() {
+    }
+
+    @Override
+    public ZNRecord update(ZNRecord current) {
+      // If we are still the lock owner, clean owner field.
+      LockInfo curLockInfo = new LockInfo(current);
+      if (isCurrentOwner(curLockInfo)) {
+        ZNRecord record = current;

Review comment:
       If I just create a new LockInfo(null) object, then I get the same result, right?
   I guess maybe you want to make the LockInfo(<no parameter>) constructor public.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,11 +222,72 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (isCurrentOwner(lockInfo)) {
+      if (lockInfo.getRequestorId().equals("NONE") || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        CleanupUpdater cleanupUpdater = new CleanupUpdater();
+        boolean res = _baseDataAccessor.update(_lockPath, cleanupUpdater, AccessOption.PERSISTENT);
+        if (!res) {
+          throw new HelixException(
+              String.format("User %s failed to update lock path %s", _userId, _lockPath));
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _lockStatus == LockConstants.LockStatus.PENDING) {
+      onDeniedPendingLockNotification();
+    }
+  }
+
+  /**
+   * call back called when the lock is acquired
+   */
+  public void onAcquiredLockNotification() {
+    _lockStatus = LockConstants.LockStatus.LOCKED;
+    _countDownLatch.countDown();
+  }
+
   /**
-   * Class that specifies how a lock node should be updated with another lock node
+   * call back called when the pending request is denied due to another higher priority request
+   */
+  public void onDeniedPendingLockNotification() {
+    _lockStatus = LockConstants.LockStatus.PREEMPTED;
+    _countDownLatch.countDown();
+  }
+
+  @Override
+  public void handleDataDeleted(String dataPath) throws Exception {
+  }

Review comment:
       Once we support LOCK node cleanup, then we need to add some logic here, right?
   Maybe some TODO comments would be helpful.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();

Review comment:
       The proposal is that the waiting time should start once cleanup starts. But that could be too complicated. So let's just add a TODO here. We shall only do this when it is proved to be required.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +305,143 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
-      LOG.error(
-          "User " + _userId + " tried to update the lock at " + new Date(System.currentTimeMillis())
-              + ". Lock path: " + _lockPath);
-      throw new HelixException("User is not authorized to perform this operation.");
+
+      // higher priority lock request will try to  preempt current lock owner
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority()) {
+        if (curLockInfo.getRequestorId().equals("NONE")) {
+          // update lock Znode with requestor information
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          _pendingTimeout =
+              _waitingTimeout > curLockInfo.getCleanupTimeout() ? curLockInfo.getCleanupTimeout()
+                  : _waitingTimeout;
+          return newRecord;
+        } // If the requestor field is not empty, and the coming lock request has a even higher
+        // priority. The new request will replace current requestor field of the lock
+        else if (_priority > curLockInfo.getRequestorPriority()) {
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          long remainingCleanupTime =
+              curLockInfo.getCleanupTimeout() - (System.currentTimeMillis() - curLockInfo
+                  .getRequestingTimestamp());
+          _pendingTimeout = _waitingTimeout > remainingCleanupTime ? remainingCleanupTime :
+           _waitingTimeout;
+          return newRecord;
+        }
+      }
+
+      // For users who are not the lock owner and the priority is not higher than current lock
+      // owner, or the priority is higher than current lock, but lower than the requestor, throw
+      // an exception. The exception will be caught by data accessor, and return false for the
+      // update operation.
+      LOG.error("User {} failed to acquire lock at Lock path {}.", _userId, _lockPath);
+      throw new HelixException(
+          String.format("User %s failed to acquire lock at Lock path %s.", _userId, _lockPath));
     }
   }
 
+  /**
+   * Class that specifies how a lock node should be updated after the previous owner finishes the
+   * cleanup work
+   */
+  private class CleanupUpdater implements DataUpdater<ZNRecord> {
+    public CleanupUpdater() {
+    }
+
+    @Override
+    public ZNRecord update(ZNRecord current) {
+      // If we are still the lock owner, clean owner field.
+      LockInfo curLockInfo = new LockInfo(current);
+      if (isCurrentOwner(curLockInfo)) {
+        ZNRecord record = current;

Review comment:
       Or even better, just define a static empty LockInfo object for this usage.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r536493729



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _isPending) {

Review comment:
       Actually we need, because this callback can also be received by previous owner (who already cleaned up the ownership), and it should not act since it's not pending.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539633523



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       np, I saw it later.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r536524837



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {

Review comment:
       I see your point. So here we cannot use while because pending is not the only checking condition. We also have timeout, which can terminate the wait. We cannot easily distinguish between spurious wake up and timeout. Using a timer is not accurate. I changed to use countdown latch, which is more convenient and doesn't have spurious wake up issue.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535866217



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,

Review comment:
       long should be fine, but this is old API. I'm concerned about backward compatibility.
   A wrapping class sounds good. But still, there's backward compatibility concern. I'll see whether I can make it look simpler.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r538942986



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _isPending) {
+      onDeniedPendingLockNotification();
+    }
+  }
+
+  /**
+   * call back called when the lock is acquired
+   */
+  public void onAcquiredLockNotification() {
+    synchronized (ZKDistributedNonblockingLock.this) {
+      _isLocked = true;

Review comment:
       Yeah, so _isPending should be false.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535871361



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);

Review comment:
       I should use LockInfo instead of directly setting it in ZNRecord. Updated.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r533672387



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);

Review comment:
       This wait() will be signaled by the following notify(), or if it never receives the notify, it'll wait until timeout.
   
     public void onAcquiredLockNotification() {
       synchronized (ZKDistributedNonblockingLock.this) {
         _isLocked = true;
         ZKDistributedNonblockingLock.this.notify();
       }
     }

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {

Review comment:
       This boolean variable is used as a flag to distinguish between different scenarios so that we can act differently. The actual functioning part is the following wait(). 

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -30,13 +30,23 @@
   // Default values for each attribute if there are no current values set by user
   public static final String DEFAULT_OWNER_TEXT = "";
   public static final String DEFAULT_MESSAGE_TEXT = "";
-  public static final long DEFAULT_TIMEOUT_LONG = -1L;
+  public static final long DEFAULT_TIMEOUT_LONG = -1;
+  public static final int DEFAULT_PRIORITY_INT = -1;
+  public static final long DEFAULT_WAITING_TIMEOUT_LONG = -1;
+  public static final long DEFAULT_CLEANUP_TIMEOUT_LONG = -1;
+  public static final String DEFAULT_REQUESTOR_ID = "";
+  public static final int DEFAULT_REQUESTOR_PRIORITY_INT = -1;
+  public static final long DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG = -1;
+  public static final long DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG = -1;

Review comment:
       Good suggestion! Will do.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r538943437



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
+
+      // higher priority lock request will preempt current lock owner that is with lower priority
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority() && curLockInfo
+          .getRequestorId().equals("NONE")) {
+        // update lock Znode with requestor information
+        ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+        _isPending = true;

Review comment:
       Isn't it receiving notification on preempting?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539046511



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       I actually added the validation logic in `ZKDistributedNonblockingLock` in the constructor. Did you suggest to add in this file? I feel it's more of a validation of customer's input.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r540571099



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -32,68 +33,167 @@
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private LockConstants.LockStatus _lockStatus;
+  private long _pendingTimeout;
+  private CountDownLatch _countDownLatch = new CountDownLatch(1);
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false, null,
+        new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
-   * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
+   * Initialize the lock with ZKLockConfig. This is the preferred way to construct the lock.
+   */
+  public ZKDistributedNonblockingLock(ZKLockConfig zkLockConfig) {
+    this(zkLockConfig.getLockScope(), zkLockConfig.getZkAddress(), zkLockConfig.getLeaseTimeout(),
+        zkLockConfig.getLockMsg(), zkLockConfig.getUserId(), zkLockConfig.getPriority(),
+        zkLockConfig.getWaitingTimeout(), zkLockConfig.getCleanupTimeout(),
+        zkLockConfig.getIsForceful(), zkLockConfig.getLockListener());
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, priority,
+   * different kinds of timeout, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  private ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Internal construction of the lock with user provided information, e.g., lock path under
+   * zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
+    validateInput();
   }
 
   @Override
   public boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Immediately return if the lock statue is not PENDING.
+    if (_lockStatus != LockConstants.LockStatus.PENDING) {

Review comment:
       Following lockStatus was protected by latch down. But this check could happening parallel to following lockStatus update. Do we need some synchronization?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539640437



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -32,68 +33,167 @@
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private LockConstants.LockStatus _lockStatus;
+  private long _pendingTimeout;
+  private CountDownLatch _countDownLatch = new CountDownLatch(1);
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false, null,
+        new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
-   * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
+   * Initialize the lock with ZKLockConfig. This is the preferred way to construct the lock.
+   */
+  public ZKDistributedNonblockingLock(ZKLockConfig zkLockConfig) {
+    this(zkLockConfig.getLockScope(), zkLockConfig.getZkAddress(), zkLockConfig.getLeaseTimeout(),
+        zkLockConfig.getLockMsg(), zkLockConfig.getUserId(), zkLockConfig.getPriority(),
+        zkLockConfig.getWaitingTimeout(), zkLockConfig.getCleanupTimeout(),
+        zkLockConfig.getIsForceful(), zkLockConfig.getLockListener());
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, priority,
+   * different kinds of timeout, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,

Review comment:
       I think we want users to use zooscalibility features as far as they can. So shall we hide this constructor? The above one with zkAddress is left because we want to ensure backward compatibility. But this one is newly added. So make it private or get rid of it?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r538942664



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();

Review comment:
       My proposal is that do the following steps in our code.
   
   1. update znode about the cleanup start time.
   2. trigger clean up callbacks.
   
   So the cleanup start time is a more accurate sign than guessing.
   I'm fine if we just add a TODO here about step 1.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535866217



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,

Review comment:
       long should be fine, but this is old API. I'm concerned about backward compatibility.
   A wrapping class sounds good. I'll give it a try.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535786915



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);
+      } catch (InterruptedException e) {
+        throw new HelixException(
+            String.format("Interruption happened while %s is waiting for the " + "lock", _userId),
+            e);
+      }
+      // We have not acquired the lock yet.
+      if (!_isLocked) {
+        // If the reason for not being able to acquire the lock is due to high priority lock
+        // preemption, directly return false.
+        if (_isPreempted) {
+          _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
+          return false;
+        } else {
+          // Forceful lock request will grab the lock even the current owner has not finished
+          // cleanup work, while non forceful lock request will get an exception.
+          if (_isForceful) {
+            ZNRecord znRecord = composeNewOwnerRecord();
+            LOG.info("Updating Zookeeper with new owner {} information", _userId);
+            _baseDataAccessor
+                .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+            return true;
+          } else {
+            _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
+            throw new HelixException("Cleanup has not been finished by lock owner");
+          }
+        }
+      }
     }
-    LockUpdater updater = new LockUpdater(new LockInfo(_userId, _lockMsg, deadline));
-    return _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+    if (!updateResult) {

Review comment:
       The isLocked is actually inside isPending condition, it checks whether the lock is acquired after the waiting timeout expired. And this updateResult is the immediate response from update operation. So it can be true or false depending on the request's priority and owner. E.g. if the request is not the owner and has a lower priority than the owner, the response will be false, and there is no pending.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {

Review comment:
       Yeah, there won't be Znode, and the input value is null.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539489963



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKLockConfig.java
##########
@@ -0,0 +1,173 @@
+package org.apache.helix.lock.helix;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.lock.LockScope;
+import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Hold configs used for a ZK distributed nonblocking lock.
+ */
+public class ZKLockConfig {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(org.apache.helix.lock.helix.ZKLockConfig.class.getName());
+  private LockScope _lockScope;
+  private String _zkAddress;

Review comment:
       Please see the above comment. I feel multi ZK should be implicitly supported. Let me know if that's not the case. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539034860



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       Sounds good 👌 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535840946



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);
+      } catch (InterruptedException e) {
+        throw new HelixException(
+            String.format("Interruption happened while %s is waiting for the " + "lock", _userId),
+            e);
+      }
+      // We have not acquired the lock yet.
+      if (!_isLocked) {

Review comment:
       This _isLocked check is actually inside _isPending. It is only checked after wait timed out. Other immediately returned results are outside _isPending. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539002977



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/LockInfo.java
##########
@@ -65,34 +77,70 @@ public LockInfo(ZNRecord znRecord) {
     if (znRecord != null) {
       String ownerId = znRecord.getSimpleField(LockInfoAttribute.OWNER.name());
       String message = znRecord.getSimpleField(LockInfoAttribute.MESSAGE.name());
-      long timeout = znRecord.getLongField(LockInfoAttribute.TIMEOUT.name(), DEFAULT_TIMEOUT_LONG);
-      setLockInfoFields(ownerId, message, timeout);
+      long timeout = znRecord
+          .getLongField(LockInfoAttribute.TIMEOUT.name(), LockConstants.DEFAULT_TIMEOUT_LONG);
+      int priority = znRecord
+          .getIntField(LockInfoAttribute.PRIORITY.name(), LockConstants.DEFAULT_PRIORITY_INT);
+      long waitingTimeout = znRecord.getLongField(LockInfoAttribute.WAITING_TIMEOUT.name(),
+          LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+      long cleanupTimeout = znRecord.getLongField(LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+          LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+      String requestorId = znRecord.getSimpleField(LockInfoAttribute.REQUESTOR_ID.name());
+      int requestorPriority = znRecord.getIntField(LockInfoAttribute.REQUESTOR_PRIORITY.name(),
+          LockConstants.DEFAULT_REQUESTOR_PRIORITY_INT);
+      long requestorWaitingTimeout = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG);
+      long requestorRequestingTimestamp = znRecord
+          .getLongField(LockInfoAttribute.REQUESTOR_REQUESTING_TIMESTAMP.name(),
+              LockConstants.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG);
+      setLockInfoFields(ownerId, message, timeout, priority, waitingTimeout, cleanupTimeout,
+          requestorId, requestorPriority, requestorWaitingTimeout, requestorRequestingTimestamp);
     }
   }
 
   /**
-   * Initialize a LockInfo with data for each field, set all null info fields to default data
+   * Initialize a LockInfo with data for each field, set all null info fields to default data.
    * @param ownerId value of OWNER attribute
    * @param message value of MESSAGE attribute
    * @param timeout value of TIMEOUT attribute
+   * @param priority value of PRIORITY attribute
+   * @param waitingTimout value of WAITING_TIMEOUT attribute
+   * @param cleanupTimeout value of CLEANUP_TIMEOUT attribute
+   * @param requestorId value of REQUESTOR_ID attribute
+   * @param requestorPriority value of REQUESTOR_PRIORITY attribute
+   * @param requestorWaitingTimeout value of REQUESTOR_WAITING_TIMEOUT attribute
+   * @param requestorRequestingTimestamp value of REQUESTOR_REQUESTING_TIMESTAMP attribute
    */
-  public LockInfo(String ownerId, String message, long timeout) {
+  public LockInfo(String ownerId, String message, long timeout, int priority, long waitingTimout,

Review comment:
       All the new input parameters added this time are optional. I added a new validation function to throw exception if required field is missing or invalid value is given.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535840946



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);
+      } catch (InterruptedException e) {
+        throw new HelixException(
+            String.format("Interruption happened while %s is waiting for the " + "lock", _userId),
+            e);
+      }
+      // We have not acquired the lock yet.
+      if (!_isLocked) {

Review comment:
       I modified this part of logic to make it simpler. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r537925046



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {

Review comment:
       Sorry. I am asking what if there is no ZNode, will this function breaking? Shall we create one? Because creating could also have race condition happening. How we handle the create fail.
   




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535731076



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);
+      } catch (InterruptedException e) {
+        throw new HelixException(
+            String.format("Interruption happened while %s is waiting for the " + "lock", _userId),
+            e);
+      }
+      // We have not acquired the lock yet.
+      if (!_isLocked) {

Review comment:
       As discussed, please double check if the logic is complete here since if islocked == false, it will return in any case without checking the updateResult.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);
+      } catch (InterruptedException e) {
+        throw new HelixException(
+            String.format("Interruption happened while %s is waiting for the " + "lock", _userId),
+            e);
+      }
+      // We have not acquired the lock yet.
+      if (!_isLocked) {
+        // If the reason for not being able to acquire the lock is due to high priority lock
+        // preemption, directly return false.
+        if (_isPreempted) {
+          _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
+          return false;
+        } else {
+          // Forceful lock request will grab the lock even the current owner has not finished
+          // cleanup work, while non forceful lock request will get an exception.
+          if (_isForceful) {
+            ZNRecord znRecord = composeNewOwnerRecord();
+            LOG.info("Updating Zookeeper with new owner {} information", _userId);
+            _baseDataAccessor
+                .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+            return true;
+          } else {
+            _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
+            throw new HelixException("Cleanup has not been finished by lock owner");
+          }
+        }
+      }
     }
-    LockUpdater updater = new LockUpdater(new LockInfo(_userId, _lockMsg, deadline));
-    return _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+    if (!updateResult) {

Review comment:
       Here we have isLocked == true and updateResult == false, is it possible?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {

Review comment:
       Can we change to isCurrentOwner(...)

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);
+      } catch (InterruptedException e) {
+        throw new HelixException(
+            String.format("Interruption happened while %s is waiting for the " + "lock", _userId),
+            e);
+      }
+      // We have not acquired the lock yet.
+      if (!_isLocked) {
+        // If the reason for not being able to acquire the lock is due to high priority lock
+        // preemption, directly return false.
+        if (_isPreempted) {
+          _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
+          return false;
+        } else {

Review comment:
       This else seems to be unnecessary.
   And we need to comment that the order there is important and shall not be changed.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data

Review comment:
       As discussed, there is still a chance that we hit a race condition here. So I will suggest we check it in the updater.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
+
+      // higher priority lock request will preempt current lock owner that is with lower priority
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority() && curLockInfo
+          .getRequestorId().equals("NONE")) {
+        // update lock Znode with requestor information
+        ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+        _isPending = true;

Review comment:
       As mentioned above, I think this would be better to set in the notification method.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _isPending) {
+      onDeniedPendingLockNotification();
+    }
+  }
+
+  /**
+   * call back called when the lock is acquired
+   */
+  public void onAcquiredLockNotification() {
+    synchronized (ZKDistributedNonblockingLock.this) {
+      _isLocked = true;
+      ZKDistributedNonblockingLock.this.notify();
+    }
+  }
+
+  /**
+   * call back called when the pending request is denied due to another higher priority request
+   */
+  public void onDeniedPendingLockNotification() {
+    synchronized (ZKDistributedNonblockingLock.this) {
+      _isLocked = false;

Review comment:
       Looks like a lock can only be in one state at a time. I suggest merging these 2 booleans into an enum for the cleanness of the code and the logic.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _isPending) {
+      onDeniedPendingLockNotification();
+    }
+  }
+
+  /**
+   * call back called when the lock is acquired
+   */
+  public void onAcquiredLockNotification() {
+    synchronized (ZKDistributedNonblockingLock.this) {
+      _isLocked = true;

Review comment:
       I think _isPending would be better to be updated here.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r538633776



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {

Review comment:
       Also regarding the above question `Are we notifying them or we give them callback and process clean up here?`
   Seems I cannot directly reply, so just answer here. We provide an interface of cleanup, and clients should implement that cleanup callback.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535733668



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _isPending) {
+      onDeniedPendingLockNotification();
+    }
+  }
+
+  /**
+   * call back called when the lock is acquired
+   */
+  public void onAcquiredLockNotification() {
+    synchronized (ZKDistributedNonblockingLock.this) {

Review comment:
       We can just put synchronized (this)

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
+
+      // higher priority lock request will preempt current lock owner that is with lower priority
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority() && curLockInfo
+          .getRequestorId().equals("NONE")) {
+        // update lock Znode with requestor information
+        ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+        _isPending = true;
+        _pendingTimeout = _waitingTimeout > curLockInfo.getCleanupTimeout() ? curLockInfo.getCleanupTimeout()
+                : _waitingTimeout;
+        return newRecord;
+      }
+
+      // If the requestor field is not empty, and the coming lock request has a even higher
+      // priority. The new request will replace current requestor field of the lock
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority() && !curLockInfo
+          .getRequestorId().equals("NONE") && _priority > curLockInfo.getRequestorPriority()) {

Review comment:
       This could be a sub branch of previous one since the previous two checks are same. Can we combine them?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
+
+      // higher priority lock request will preempt current lock owner that is with lower priority
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority() && curLockInfo
+          .getRequestorId().equals("NONE")) {
+        // update lock Znode with requestor information
+        ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+        _isPending = true;
+        _pendingTimeout = _waitingTimeout > curLockInfo.getCleanupTimeout() ? curLockInfo.getCleanupTimeout()
+                : _waitingTimeout;
+        return newRecord;
+      }
+
+      // If the requestor field is not empty, and the coming lock request has a even higher
+      // priority. The new request will replace current requestor field of the lock
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority() && !curLockInfo
+          .getRequestorId().equals("NONE") && _priority > curLockInfo.getRequestorPriority()) {
+        ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+        _isPending = true;
+        long remainingCleanupTime =
+            curLockInfo.getCleanupTimeout() - (System.currentTimeMillis() - curLockInfo
+                .getRequestorRequestingTimestamp());
+        _pendingTimeout =
+            _waitingTimeout > remainingCleanupTime ? remainingCleanupTime : _waitingTimeout;
+        return newRecord;
+      }
+
+      // For users who are not the lock owner and the priority is not higher than current lock
+      // owner, throw an exception. The exception will be caught by data accessor, and return
+      // false for the update
       LOG.error(
           "User " + _userId + " tried to update the lock at " + new Date(System.currentTimeMillis())
               + ". Lock path: " + _lockPath);
+
       throw new HelixException("User is not authorized to perform this operation.");

Review comment:
       The message could be confusing here. We can say the user has failed to acquire the lock. 

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);

Review comment:
       Can we combine the logic with composeRecordOwner? Let it can pass arguments. Then we can simplify the code. Otherwise, too many places setting the fields.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +314,91 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {

Review comment:
       What if there is no owner at first beginning? Even no ZNode exist?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;

Review comment:
       I would suggest use an enum to represent its current state:
   
   LOCKED, PENDING, PREEMPTED. Otherwise, three flags can be confusing for mutual usage to present the lock state.

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();

Review comment:
       Are we notifying them or we give them callback and process clean up here?

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -122,9 +210,91 @@ public void close() {
     if (isCurrentOwner()) {
       throw new HelixException("Please unlock the lock before closing it.");
     }
+    _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId().equals("NONE")|| lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.debug("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();
+        // read the lock information again to avoid stale data
+        readData = _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+        // If we are still the lock owner, clean the lock owner field.
+        if (lockInfo.getOwner().equals(_userId)) {
+          ZNRecord znRecord = new ZNRecord(readData);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.OWNER.name(),
+              LockConstants.DEFAULT_OWNER_TEXT);
+          znRecord.setSimpleField(LockInfo.LockInfoAttribute.MESSAGE.name(),
+              LockConstants.DEFAULT_MESSAGE_TEXT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.TIMEOUT.name(),
+              LockConstants.DEFAULT_TIMEOUT_LONG);
+          znRecord.setIntField(LockInfo.LockInfoAttribute.PRIORITY.name(),
+              LockConstants.DEFAULT_PRIORITY_INT);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.WAITING_TIMEOUT.name(),
+              LockConstants.DEFAULT_WAITING_TIMEOUT_LONG);
+          znRecord.setLongField(LockInfo.LockInfoAttribute.CLEANUP_TIMEOUT.name(),
+              LockConstants.DEFAULT_CLEANUP_TIMEOUT_LONG);
+          _baseDataAccessor
+              .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+        } else {
+          LOG.info("We are not current lock owner");
+        }
+      }
+    } // We are the current requestor
+    else if (lockInfo.getRequestorId().equals(_userId)) {
+      // In case the owner field is empty, it means previous owner has finished cleanup work.
+      if (lockInfo.getOwner().equals("NONE")) {
+        ZNRecord znRecord = composeNewOwnerRecord();
+        LockUpdater updater = new LockUpdater(new LockInfo(znRecord));
+        _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+        onAcquiredLockNotification();
+      } else {
+        LOG.info("We do not need to handle this data change");
+      }
+    } // If we are waiting for the lock, but find we are not the requestor any more, meaning we
+    // are preempted by an even higher priority request
+    else if (!lockInfo.getRequestorId().equals("NONE") && !lockInfo.getRequestorId().equals(_userId)
+        && _isPending) {

Review comment:
       Do we need to check this _isPending? I think we can notify anyway if it is pending or not pending. 

##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {

Review comment:
       Java actually has spurious wake up. If you dont have while, then you may not wait until timeout reaches and failed to acquire the lock. That's the reason of using while.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r542604466



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -32,68 +33,167 @@
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private LockConstants.LockStatus _lockStatus;
+  private long _pendingTimeout;
+  private CountDownLatch _countDownLatch = new CountDownLatch(1);
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false, null,
+        new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
-   * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
+   * Initialize the lock with ZKLockConfig. This is the preferred way to construct the lock.
+   */
+  public ZKDistributedNonblockingLock(ZKLockConfig zkLockConfig) {
+    this(zkLockConfig.getLockScope(), zkLockConfig.getZkAddress(), zkLockConfig.getLeaseTimeout(),
+        zkLockConfig.getLockMsg(), zkLockConfig.getUserId(), zkLockConfig.getPriority(),
+        zkLockConfig.getWaitingTimeout(), zkLockConfig.getCleanupTimeout(),
+        zkLockConfig.getIsForceful(), zkLockConfig.getLockListener());
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, priority,
+   * different kinds of timeout, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  private ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Internal construction of the lock with user provided information, e.g., lock path under
+   * zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
+    validateInput();
   }
 
   @Override
   public boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Immediately return if the lock statue is not PENDING.
+    if (_lockStatus != LockConstants.LockStatus.PENDING) {

Review comment:
       Just make sure one thing, this is not shared status between two calls of same path. Then it should be safe for this case.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535786915



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,157 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        null, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);
+      } catch (InterruptedException e) {
+        throw new HelixException(
+            String.format("Interruption happened while %s is waiting for the " + "lock", _userId),
+            e);
+      }
+      // We have not acquired the lock yet.
+      if (!_isLocked) {
+        // If the reason for not being able to acquire the lock is due to high priority lock
+        // preemption, directly return false.
+        if (_isPreempted) {
+          _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
+          return false;
+        } else {
+          // Forceful lock request will grab the lock even the current owner has not finished
+          // cleanup work, while non forceful lock request will get an exception.
+          if (_isForceful) {
+            ZNRecord znRecord = composeNewOwnerRecord();
+            LOG.info("Updating Zookeeper with new owner {} information", _userId);
+            _baseDataAccessor
+                .update(_lockPath, new ZNRecordUpdater(znRecord), AccessOption.PERSISTENT);
+            return true;
+          } else {
+            _baseDataAccessor.unsubscribeDataChanges(_lockPath, this);
+            throw new HelixException("Cleanup has not been finished by lock owner");
+          }
+        }
+      }
     }
-    LockUpdater updater = new LockUpdater(new LockInfo(_userId, _lockMsg, deadline));
-    return _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+    if (!updateResult) {

Review comment:
       Updated.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r535843744



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();

Review comment:
       Clients need to implement the cleanup callback in their logic. We only trigger the registered listener. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539006984



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -125,6 +211,79 @@ public void close() {
     _baseDataAccessor.close();
   }
 
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    Stat stat = new Stat();
+    ZNRecord readData =
+        _baseDataAccessor.get(dataPath, stat, AccessOption.THROW_EXCEPTION_IFNOTEXIST);
+    LockInfo lockInfo = new LockInfo(readData);
+    // We are the current owner
+    if (lockInfo.getOwner().equals(_userId)) {
+      if (lockInfo.getRequestorId() == null || lockInfo.getPriority() > lockInfo
+          .getRequestorPriority()) {
+        LOG.info("We do not need to handle this data change");
+      } else {
+        _lockListener.onCleanupNotification();

Review comment:
       I agree with you that current time calculation is not accurate enough. But even with recording cleanup start time in znode, we still have a problem letting the requestor know the start time and adjust its waiting. Right now, the waiting time is calculated when requestor updates the znode, and cleanup update znode happens after requestor updates znode. How can we adjust the waiting time for requestor in this case? 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r539489215



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -144,19 +305,143 @@ public ZNRecord update(ZNRecord current) {
       // If no one owns the lock, allow the update
       // If the user is the current lock owner, allow the update
       LockInfo curLockInfo = new LockInfo(current);
-      if (!(System.currentTimeMillis() < curLockInfo.getTimeout()) || isCurrentOwner()) {
+      if (System.currentTimeMillis() > curLockInfo.getTimeout() || isCurrentOwner(curLockInfo)) {
         return _record;
       }
-      // For users who are not the lock owner and try to do an update on a lock that is held by
-      // someone else, exception thrown is to be caught by data accessor, and return false for
-      // the update
-      LOG.error(
-          "User " + _userId + " tried to update the lock at " + new Date(System.currentTimeMillis())
-              + ". Lock path: " + _lockPath);
-      throw new HelixException("User is not authorized to perform this operation.");
+
+      // higher priority lock request will try to  preempt current lock owner
+      if (!isCurrentOwner(curLockInfo) && _priority > curLockInfo.getPriority()) {
+        if (curLockInfo.getRequestorId().equals("NONE")) {
+          // update lock Znode with requestor information
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          _pendingTimeout =
+              _waitingTimeout > curLockInfo.getCleanupTimeout() ? curLockInfo.getCleanupTimeout()
+                  : _waitingTimeout;
+          return newRecord;
+        } // If the requestor field is not empty, and the coming lock request has a even higher
+        // priority. The new request will replace current requestor field of the lock
+        else if (_priority > curLockInfo.getRequestorPriority()) {
+          ZNRecord newRecord = composeNewRequestorRecord(curLockInfo, _record);
+          _lockStatus = LockConstants.LockStatus.PENDING;
+          long remainingCleanupTime =
+              curLockInfo.getCleanupTimeout() - (System.currentTimeMillis() - curLockInfo
+                  .getRequestingTimestamp());
+          _pendingTimeout = _waitingTimeout > remainingCleanupTime ? remainingCleanupTime :
+           _waitingTimeout;
+          return newRecord;
+        }
+      }
+
+      // For users who are not the lock owner and the priority is not higher than current lock
+      // owner, or the priority is higher than current lock, but lower than the requestor, throw
+      // an exception. The exception will be caught by data accessor, and return false for the
+      // update operation.
+      LOG.error("User {} failed to acquire lock at Lock path {}.", _userId, _lockPath);
+      throw new HelixException(
+          String.format("User %s failed to acquire lock at Lock path %s.", _userId, _lockPath));
     }
   }
 
+  /**
+   * Class that specifies how a lock node should be updated after the previous owner finishes the
+   * cleanup work
+   */
+  private class CleanupUpdater implements DataUpdater<ZNRecord> {
+    public CleanupUpdater() {
+    }
+
+    @Override
+    public ZNRecord update(ZNRecord current) {
+      // If we are still the lock owner, clean owner field.
+      LockInfo curLockInfo = new LockInfo(current);
+      if (isCurrentOwner(curLockInfo)) {
+        ZNRecord record = current;

Review comment:
       Here we only clean up the owner related fields, but still contain requestor related fields, so that requestor can go ahead to acquire the lock. So we need to get the current lock and update based on that lock, instead of creating a default lock node.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1564: Implement Helix lock priority and notification

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1564:
URL: https://github.com/apache/helix/pull/1564#discussion_r533672387



##########
File path: helix-lock/src/main/java/org/apache/helix/lock/helix/ZKDistributedNonblockingLock.java
##########
@@ -31,69 +31,162 @@
 import org.apache.helix.manager.zk.GenericZkHelixApiBuilder;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.ZNRecordUpdater;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
-import org.apache.log4j.Logger;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.lock.LockInfo.DEFAULT_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_ID;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_PRIORITY_INT;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_REQUESTING_TIMESTAMP_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_REQUESTOR_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.DEFAULT_WAITING_TIMEOUT_LONG;
+import static org.apache.helix.lock.LockInfo.ZNODE_ID;
 
 
 /**
  * Helix nonblocking lock implementation based on Zookeeper.
  * NOTE: do NOT use ephemeral nodes in this implementation because ephemeral mode is not supported
  * in ZooScalability mode.
  */
-public class ZKDistributedNonblockingLock implements DistributedLock {
-  private static final Logger LOG = Logger.getLogger(ZKDistributedNonblockingLock.class);
+public class ZKDistributedNonblockingLock implements DistributedLock, IZkDataListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ZKDistributedNonblockingLock.class);
 
   private final String _lockPath;
   private final String _userId;
-  private final long _timeout;
   private final String _lockMsg;
+  private final long _leaseTimeout;
+  private final long _waitingTimeout;
+  private final long _cleanupTimeout;
+  private final int _priority;
+  private final boolean _isForceful;
+  private final LockListener _lockListener;
   private final BaseDataAccessor<ZNRecord> _baseDataAccessor;
+  private boolean _isLocked;
+  private boolean _isPending;
+  private boolean _isPreempted;
+  private long _pendingTimeout;
 
   /**
    * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
    * @param scope the scope to lock
    * @param zkAddress the zk address the cluster connects to
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
    */
-  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long timeout,
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
       String lockMsg, String userId) {
-    this(scope.getPath(), timeout, lockMsg, userId, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, 0, Integer.MAX_VALUE, 0, false,
+        new LockListener() {
+          @Override
+          public void onCleanupNotification() {
+          }
+        }, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
+  }
+
+  /**
+   * Initialize the lock with user provided information, e.g.,cluster, scope, etc.
+   * @param scope the scope to lock
+   * @param zkAddress the zk address the cluster connects to
+   * @param leaseTimeout the leasing timeout period of the lock
+   * @param lockMsg the reason for having this lock
+   * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
+   */
+  public ZKDistributedNonblockingLock(LockScope scope, String zkAddress, Long leaseTimeout,
+      String lockMsg, String userId, int priority, long waitingTimeout, long cleanupTimeout,
+      boolean isForceful, LockListener lockListener) {
+    this(scope.getPath(), leaseTimeout, lockMsg, userId, priority, waitingTimeout, cleanupTimeout,
+        isForceful, lockListener, new ZkBaseDataAccessor<ZNRecord>(zkAddress));
   }
 
   /**
    * Initialize the lock with user provided information, e.g., lock path under zookeeper, etc.
    * @param lockPath the path of the lock under Zookeeper
-   * @param timeout the timeout period of the lock
+   * @param leaseTimeout the leasing timeout period of the lock
    * @param lockMsg the reason for having this lock
    * @param userId a universal unique userId for lock owner identity
+   * @param priority the priority of the lock
+   * @param waitingTimeout the waiting timeout period of the lock when the tryLock request is issued
+   * @param cleanupTimeout the time period needed to finish the cleanup work by the lock when it
+   *                      is preempted
+   * @param isForceful whether the lock is a forceful one. This determines the behavior when the
+   *                   lock encountered an exception during preempting lower priority lock
+   * @param lockListener the listener associated to the lock
    * @param baseDataAccessor baseDataAccessor instance to do I/O against ZK with
    */
-  private ZKDistributedNonblockingLock(String lockPath, Long timeout, String lockMsg, String userId,
-      BaseDataAccessor<ZNRecord> baseDataAccessor) {
+  private ZKDistributedNonblockingLock(String lockPath, Long leaseTimeout, String lockMsg,
+      String userId, int priority, long waitingTimeout, long cleanupTimeout, boolean isForceful,
+      LockListener lockListener, BaseDataAccessor<ZNRecord> baseDataAccessor) {
     _lockPath = lockPath;
-    if (timeout < 0) {
-      throw new IllegalArgumentException("The expiration time cannot be negative.");
+    if (leaseTimeout < 0 || waitingTimeout < 0 || cleanupTimeout < 0) {
+      throw new IllegalArgumentException("Timeout cannot be negative.");
+    }
+    if (priority < 0) {
+      throw new IllegalArgumentException("Priority cannot be negative.");
     }
-    _timeout = timeout;
+    _leaseTimeout = leaseTimeout;
     _lockMsg = lockMsg;
     _userId = userId;
     _baseDataAccessor = baseDataAccessor;
+    _priority = priority;
+    _waitingTimeout = waitingTimeout;
+    _cleanupTimeout = cleanupTimeout;
+    _lockListener = lockListener;
+    _isForceful = isForceful;
   }
 
   @Override
-  public boolean tryLock() {
+  public synchronized boolean tryLock() {
     // Set lock information fields
-    long deadline;
-    // Prevent value overflow
-    if (_timeout > Long.MAX_VALUE - System.currentTimeMillis()) {
-      deadline = Long.MAX_VALUE;
-    } else {
-      deadline = System.currentTimeMillis() + _timeout;
+    _baseDataAccessor.subscribeDataChanges(_lockPath, this);
+    LockUpdater updater = new LockUpdater(
+        new LockInfo(_userId, _lockMsg, getNonOverflowTimestamp(_leaseTimeout), _priority,
+            _waitingTimeout, _cleanupTimeout, null, 0, 0, 0));
+    boolean updateResult = _baseDataAccessor.update(_lockPath, updater, AccessOption.PERSISTENT);
+
+    // Check whether the lock request is still pending. If yes, we will wait for the period
+    // recorded in _pendingTimeout.
+    if (_isPending) {
+      try {
+        wait(_pendingTimeout);

Review comment:
       This wait() will be signaled by the following notify(), or if it never receives the notify, it'll wait until timeout.
   
    ` public void onAcquiredLockNotification() {
       synchronized (ZKDistributedNonblockingLock.this) {
         _isLocked = true;
         ZKDistributedNonblockingLock.this.notify();
       }
     }`




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org