You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2017/08/15 23:10:48 UTC

[37/50] [abbrv] geode git commit: GEODE-3314 - Refactoring of DLockService to improve developer QoL. This now closes #683

GEODE-3314 - Refactoring of DLockService to improve developer QoL. This now closes #683

* Write characterization tests for DLockService.
* Remove debugging code.
* Remove dead code.
* Remove comments.
* Extract the local lock granting into a separate function.

Between the characterization tests we've written and the existing DUnit
tests, the coverage should be fairly adequate.

Signed-off-by: Hitesh Khamesra <hk...@pivotal.io>
Signed-off-by: Galen O'Sullivan <go...@pivotal.io>


Project: http://git-wip-us.apache.org/repos/asf/geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/geode/commit/190cfed8
Tree: http://git-wip-us.apache.org/repos/asf/geode/tree/190cfed8
Diff: http://git-wip-us.apache.org/repos/asf/geode/diff/190cfed8

Branch: refs/heads/feature/GEODE-3249
Commit: 190cfed880da17b947eb520948866062b9aafe0b
Parents: a3c0eba
Author: Galen O'Sullivan <go...@pivotal.io>
Authored: Wed Aug 2 11:29:21 2017 -0700
Committer: Udo Kohlmeyer <uk...@pivotal.io>
Committed: Tue Aug 15 10:08:40 2017 -0700

----------------------------------------------------------------------
 .../internal/locks/DLockRequestProcessor.java   |   7 +
 .../internal/locks/DLockService.java            | 284 +++++--------------
 .../distributed/internal/locks/DLockToken.java  |  12 +-
 .../DLockServiceCharacterizationTests.java      | 124 ++++++++
 4 files changed, 211 insertions(+), 216 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/190cfed8/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRequestProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRequestProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRequestProcessor.java
index 3f42adb..96f692b 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRequestProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRequestProcessor.java
@@ -196,6 +196,13 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
     return this.response.leaseExpireTime;
   }
 
+  /**
+   *
+   * @param interruptible
+   * @param lockId
+   * @return
+   * @throws InterruptedException only possible if interruptible is true.
+   */
   protected boolean requestLock(boolean interruptible, int lockId) throws InterruptedException {
     final boolean isDebugEnabled_DLS = logger.isTraceEnabled(LogMarker.DLS);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/190cfed8/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
index 522b700..f0377b4 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
@@ -17,7 +17,6 @@ package org.apache.geode.distributed.internal.locks;
 
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.CancelException;
-import org.apache.geode.InternalGemFireError;
 import org.apache.geode.InternalGemFireException;
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.SystemFailure;
@@ -74,17 +73,6 @@ public class DLockService extends DistributedLockService {
   public static final long NOT_GRANTOR_SLEEP = Long
       .getLong(DistributionConfig.GEMFIRE_PREFIX + "DLockService.notGrantorSleep", 100).longValue();
 
-  public static final boolean DEBUG_DISALLOW_NOT_HOLDER = Boolean
-      .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "DLockService.debug.disallowNotHolder");
-
-  public static final boolean DEBUG_LOCK_REQUEST_LOOP = Boolean
-      .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "DLockService.debug.disallowLockRequestLoop");
-
-  public static final int DEBUG_LOCK_REQUEST_LOOP_COUNT = Integer
-      .getInteger(
-          DistributionConfig.GEMFIRE_PREFIX + "DLockService.debug.disallowLockRequestLoopCount", 20)
-      .intValue();
-
   public static final boolean DEBUG_NONGRANTOR_DESTROY_LOOP = Boolean
       .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "DLockService.debug.nonGrantorDestroyLoop");
 
@@ -93,9 +81,6 @@ public class DLockService extends DistributedLockService {
           DistributionConfig.GEMFIRE_PREFIX + "DLockService.debug.nonGrantorDestroyLoopCount", 20)
       .intValue();
 
-  public static final boolean DEBUG_ENFORCE_SAFE_EXIT =
-      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "DLockService.debug.enforceSafeExit");
-
   public static final boolean AUTOMATE_FREE_RESOURCES =
       Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "DLockService.automateFreeResources");
 
@@ -1381,16 +1366,12 @@ public class DLockService extends DistributedLockService {
       final boolean disallowReentrant, final boolean disableAlerts) throws InterruptedException {
     checkDestroyed();
 
-    final boolean isDebugEnabled_DLS = logger.isTraceEnabled(LogMarker.DLS);
-
     boolean interrupted = Thread.interrupted();
     if (interrupted && interruptible) {
       throw new InterruptedException();
     }
 
-    boolean safeExit = true;
-    try { // try-block for abnormalExit and safeExit
-
+    try {
       long statStart = getStats().startLockWait();
       long startTime = getLockTimeStamp(dm);
 
@@ -1408,9 +1389,7 @@ public class DLockService extends DistributedLockService {
       if (waitLimit < 0)
         waitLimit = Long.MAX_VALUE;
 
-      if (isDebugEnabled_DLS) {
-        logger.trace(LogMarker.DLS, "{}, name: {} - entering lock()", this, name);
-      }
+      logger.trace(LogMarker.DLS, "{}, name: {} - entering lock()", this, name);
 
       DLockToken token = getOrCreateToken(name);
       boolean gotLock = false;
@@ -1433,29 +1412,7 @@ public class DLockService extends DistributedLockService {
         int lockId = -1;
         incActiveLocks();
 
-        int loopCount = 0;
         while (keepTrying) {
-          if (DEBUG_LOCK_REQUEST_LOOP) {
-            loopCount++;
-            if (loopCount > DEBUG_LOCK_REQUEST_LOOP_COUNT) {
-              Integer count = Integer.valueOf(DEBUG_LOCK_REQUEST_LOOP_COUNT);
-              String s =
-                  LocalizedStrings.DLockService_DEBUG_LOCKINTERRUPTIBLY_HAS_GONE_HOT_AND_LOOPED_0_TIMES
-                      .toLocalizedString(count);
-
-              InternalGemFireError e = new InternalGemFireError(s);
-              logger.error(LogMarker.DLS,
-                  LocalizedMessage.create(
-                      LocalizedStrings.DLockService_DEBUG_LOCKINTERRUPTIBLY_HAS_GONE_HOT_AND_LOOPED_0_TIMES,
-                      count),
-                  e);
-              throw e;
-            }
-            /*
-             * if (loopCount > 1) { Thread.sleep(1000); }
-             */
-          }
-
           checkDestroyed();
           interrupted = Thread.interrupted() || interrupted; // clear
           if (interrupted && interruptible) {
@@ -1469,10 +1426,8 @@ public class DLockService extends DistributedLockService {
           synchronized (token) {
             token.checkForExpiration();
             if (token.isLeaseHeldByCurrentThread()) {
-              if (isDebugEnabled_DLS) {
-                logger.trace(LogMarker.DLS, "{} , name: {} - lock() is reentrant: {}", this, name,
-                    token);
-              }
+              logger.trace(LogMarker.DLS, "{} , name: {} - lock() is reentrant: {}", this, name,
+                  token);
               reentrant = true;
               if (reentrant && disallowReentrant) {
                 throw new IllegalStateException(
@@ -1480,8 +1435,6 @@ public class DLockService extends DistributedLockService {
                         .toLocalizedString(new Object[] {Thread.currentThread(), token}));
               }
               recursionBefore = token.getRecursion();
-              leaseExpireTime = token.getLeaseExpireTime(); // moved here from processor null-check
-                                                            // under gotLock
               lockId = token.getLeaseId(); // keep lockId
               if (lockId < 0) {
                 // loop back around due to expiration
@@ -1500,156 +1453,48 @@ public class DLockService extends DistributedLockService {
             lockId = -1; // reset lockId back to -1
           }
 
-          DLockRequestProcessor processor = null;
-
-          // if reentrant w/ infinite lease TODO: remove false to restore this...
-          if (false && reentrant && leaseTimeMillis == Long.MAX_VALUE) {
-            // Optimization:
-            // thread is reentering lock and lease time is infinite so no
-            // need to trouble the poor grantor
-            gotLock = true;
-            // check for race condition...
-            Assert.assertTrue(token.isLeaseHeldByCurrentThread());
-          }
-
-          // non-reentrant or reentrant w/ non-infinite lease
-          else {
-            processor = createRequestProcessor(theLockGrantorId, name, threadId, startTime,
-                requestLeaseTime, requestWaitTime, reentrant, tryLock, disableAlerts);
-            if (reentrant) {
-              // check for race condition... reentrant expired already...
-              // related to bug 32765, but client-side... see bug 33402
-              synchronized (token) {
-                if (!token.isLeaseHeldByCurrentThread()) {
-                  reentrant = false;
-                  recursionBefore = -1;
-                  token.checkForExpiration();
-                }
+          DLockRequestProcessor processor = createRequestProcessor(theLockGrantorId, name, threadId,
+              startTime, requestLeaseTime, requestWaitTime, reentrant, tryLock, disableAlerts);
+          if (reentrant) {
+            // check for race condition... reentrant expired already...
+            // related to bug 32765, but client-side... see bug 33402
+            synchronized (token) {
+              if (!token.isLeaseHeldByCurrentThread()) {
+                reentrant = false;
+                recursionBefore = -1;
+                token.checkForExpiration();
               }
-            } else {
-              // set lockId since this is the first granting (non-reentrant)
-              lockId = processor.getProcessorId();
             }
+          } else {
+            // set lockId since this is the first granting (non-reentrant)
+            lockId = processor.getProcessorId();
+          }
 
-            try {
-              safeExit = false;
-              gotLock = processor.requestLock(interruptible, lockId);
-            } catch (InterruptedException e) { // LOST INTERRUPT
-              if (interruptible) {
-                // TODO: BUG 37158: this can cause a stuck lock
-                throw e;
-              } else {
-                interrupted = true;
-                Assert.assertTrue(false,
-                    "Non-interruptible lock is trying to throw InterruptedException");
-              }
-            }
-            if (isDebugEnabled_DLS) {
-              logger.trace(LogMarker.DLS, "Grantor {} replied {}", theLockGrantorId,
-                  processor.getResponseCodeString());
-            }
-          } // else: non-reentrant or reentrant w/ non-infinite lease
+          gotLock = processor.requestLock(interruptible, lockId); // can throw
+                                                                  // InterruptedException
+
+          logger.trace(LogMarker.DLS, "Grantor {} replied {}", theLockGrantorId,
+              processor.getResponseCodeString());
 
           if (gotLock) {
-            // if (processor != null) (cannot be null)
-            { // TODO: can be null after restoring above optimization
-              // non-reentrant lock needs to getLeaseExpireTime
-              leaseExpireTime = processor.getLeaseExpireTime();
-            }
+            leaseExpireTime = processor.getLeaseExpireTime();
             int recursion = recursionBefore + 1;
 
-            boolean granted = false;
-            boolean needToReleaseOrphanedGrant = false;
-
-            Assert.assertHoldsLock(this.destroyLock, false);
-            synchronized (this.lockGrantorIdLock) {
-              if (!checkLockGrantorId(theLockGrantorId)) {
-                safeExit = true;
-                // race: grantor changed
-                if (isDebugEnabled_DLS) {
-                  logger.trace(LogMarker.DLS,
-                      "Cannot honor grant from {} because {} is now a grantor.", theLockGrantorId,
-                      this.lockGrantorId);
-                }
-                continue;
-              } else if (isDestroyed()) {
-                // race: dls was destroyed
-                if (isDebugEnabled_DLS) {
-                  logger.trace(LogMarker.DLS,
-                      "Cannot honor grant from {} because this lock service has been destroyed.",
-                      theLockGrantorId);
-                }
-                needToReleaseOrphanedGrant = true;
-              } else {
-                safeExit = true;
-                synchronized (this.tokens) {
-                  checkDestroyed();
-                  Assert.assertTrue(token == basicGetToken(name));
-                  RemoteThread rThread =
-                      new RemoteThread(getDistributionManager().getId(), threadId);
-                  granted = token.grantLock(leaseExpireTime, lockId, recursion, rThread);
-                } // tokens sync
-              }
-            }
-
-            if (needToReleaseOrphanedGrant /* && processor != null */) {
-              processor.getResponse().releaseOrphanedGrant(this.dm);
-              safeExit = true;
+            if (!grantLocalDLockAfterObtainingRemoteLock(name, token, threadId, leaseExpireTime,
+                lockId, theLockGrantorId, processor, recursion)) {
               continue;
             }
 
-            if (!granted) {
-              Assert.assertTrue(granted, "Failed to perform client-side granting on " + token
-                  + " which was granted by " + theLockGrantorId);
-            }
-
-            // make sure token is THE instance in the map to avoid race with
-            // freeResources... ok to overwrite a newer instance too since only
-            // one thread will own the lock at a time
-            // synchronized (tokens) { // TODO: verify if this is needed
-            // synchronized (token) {
-            // if (tokens.put(name, token) == null) {
-            // getStats().incTokens(1);
-            // }
-            // }
-            // }
-
-            if (isDebugEnabled_DLS) {
-              logger.trace(LogMarker.DLS, "{}, name: {} - granted lock: {}", this, name, token);
-            }
+            logger.trace(LogMarker.DLS, "{}, name: {} - granted lock: {}", this, name, token);
             keepTrying = false;
-          } // gotLock is true
-
-          // grantor replied destroyed (getLock is false)
-          else if (processor.repliedDestroyed()) {
-            safeExit = true;
-            checkDestroyed();
-            // should have thrown LockServiceDestroyedException
+          } else if (processor.repliedDestroyed()) {
+            checkDestroyed(); // throws LockServiceDestroyedException
             Assert.assertTrue(isDestroyed(),
                 "Grantor reports service " + this + " is destroyed: " + name);
-          } // grantor replied destroyed
-
-          // grantor replied NOT_GRANTOR or departed (getLock is false)
-          else if (processor.repliedNotGrantor() || processor.hadNoResponse()) {
-            safeExit = true;
+          } else if (processor.repliedNotGrantor() || processor.hadNoResponse()) {
             notLockGrantorId(theLockGrantorId, 0, TimeUnit.MILLISECONDS);
             // keepTrying is still true... loop back around
-          } // grantor replied NOT_GRANTOR or departed
-
-          // grantor replied NOT_HOLDER for reentrant lock (getLock is false)
-          else if (processor.repliedNotHolder()) {
-            safeExit = true;
-            if (DEBUG_DISALLOW_NOT_HOLDER) {
-              String s = LocalizedStrings.DLockService_DEBUG_GRANTOR_REPORTS_NOT_HOLDER_FOR_0
-                  .toLocalizedString(token);
-              InternalGemFireError e = new InternalGemFireError(s);
-              logger.error(LogMarker.DLS,
-                  LocalizedMessage.create(
-                      LocalizedStrings.DLockService_DEBUG_GRANTOR_REPORTS_NOT_HOLDER_FOR_0, token),
-                  e);
-              throw e;
-            }
-
+          } else if (processor.repliedNotHolder()) {
             // fix part of bug 32765 - reentrant/expiration problem
             // probably expired... try to get non-reentrant lock
             reentrant = false;
@@ -1675,7 +1520,6 @@ public class DLockService extends DistributedLockService {
 
           // TODO: figure out when this else case can actually happen...
           else {
-            safeExit = true;
             // either dlock service is suspended or tryLock failed
             // fixed the math here... bug 32765
             if (waitLimit > token.getCurrentTime() + 20) {
@@ -1685,10 +1529,8 @@ public class DLockService extends DistributedLockService {
           }
 
         } // while (keepTrying)
-      } // try-block for end stats, token cleanup, and interrupt check
-
-      // finally-block for end stats, token cleanup, and interrupt check
-      finally {
+          // try-block for end stats, token cleanup, and interrupt check
+      } finally {
         getStats().endLockWait(statStart, gotLock);
 
         // cleanup token if failed to get lock
@@ -1711,26 +1553,50 @@ public class DLockService extends DistributedLockService {
         blockedOn.set(null);
       }
 
-      if (isDebugEnabled_DLS) {
-        logger.trace(LogMarker.DLS, "{}, name: {} - exiting lock() returning {}", this, name,
-            gotLock);
-      }
+      logger.trace(LogMarker.DLS, "{}, name: {} - exiting lock() returning {}", this, name,
+          gotLock);
       return gotLock;
-    } // try-block for abnormalExit and safeExit
-
-    // finally-block for abnormalExit and safeExit
-    finally {
-      if (isDebugEnabled_DLS) {
-        logger.trace(LogMarker.DLS, "{}, name: {} - exiting lock() without returning value", this,
-            name);
-      }
+    } finally {
+      logger.trace(LogMarker.DLS, "{}, name: {} - exiting lock() without returning value", this,
+          name);
       if (interrupted) {
         Thread.currentThread().interrupt();
       }
-      if (DEBUG_ENFORCE_SAFE_EXIT) {
-        Assert.assertTrue(safeExit);
+    }
+  }
+
+  private boolean grantLocalDLockAfterObtainingRemoteLock(Object name, DLockToken token,
+      int threadId, long leaseExpireTime, int lockId, LockGrantorId theLockGrantorId,
+      DLockRequestProcessor processor, int recursion) {
+    boolean needToReleaseOrphanedGrant = false;
+
+    Assert.assertHoldsLock(this.destroyLock, false);
+    synchronized (this.lockGrantorIdLock) {
+      if (!checkLockGrantorId(theLockGrantorId)) {
+        // race: grantor changed
+        logger.trace(LogMarker.DLS, "Cannot honor grant from {} because {} is now a grantor.",
+            theLockGrantorId, this.lockGrantorId);
+      } else if (isDestroyed()) {
+        // race: dls was destroyed
+        logger.trace(LogMarker.DLS,
+            "Cannot honor grant from {} because this lock service has been destroyed.",
+            theLockGrantorId);
+        needToReleaseOrphanedGrant = true;
+      } else {
+        synchronized (this.tokens) {
+          checkDestroyed();
+          Assert.assertTrue(token == basicGetToken(name));
+          RemoteThread rThread = new RemoteThread(getDistributionManager().getId(), threadId);
+          token.grantLock(leaseExpireTime, lockId, recursion, rThread);
+          return true;
+        } // tokens sync
       }
     }
+
+    if (needToReleaseOrphanedGrant) {
+      processor.getResponse().releaseOrphanedGrant(this.dm);
+    }
+    return false;
   }
 
   /**
@@ -2547,11 +2413,11 @@ public class DLockService extends DistributedLockService {
   /**
    * Called by grantor recovery to return set of locks held by this process. Synchronizes on
    * lockGrantorIdLock, tokens map, and each lock token.
-   * 
+   *
    * @param newlockGrantorId the newly recovering grantor
    */
-  Set getLockTokensForRecovery(LockGrantorId newlockGrantorId) {
-    Set heldLockSet = Collections.EMPTY_SET;
+  Set<DLockRemoteToken> getLockTokensForRecovery(LockGrantorId newlockGrantorId) {
+    Set<DLockRemoteToken> heldLockSet = Collections.EMPTY_SET;
 
     LockGrantorId currentLockGrantorId = null;
     synchronized (this.lockGrantorIdLock) {
@@ -2589,7 +2455,7 @@ public class DLockService extends DistributedLockService {
               // add token to heldLockSet
               else {
                 if (heldLockSet == Collections.EMPTY_SET) {
-                  heldLockSet = new HashSet();
+                  heldLockSet = new HashSet<>();
                 }
                 heldLockSet.add(DLockRemoteToken.createFromDLockToken(token));
               }

http://git-wip-us.apache.org/repos/asf/geode/blob/190cfed8/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockToken.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockToken.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockToken.java
index c67de67..3e85171 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockToken.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockToken.java
@@ -87,7 +87,8 @@ public class DLockToken {
   private Thread thread;
 
   /**
-   * Number of threads currently using this lock token.
+   * Number of usages of this lock token. usageCount = recursion + (# of threads waiting for this
+   * lock). It's weird, I know.
    */
   private int usageCount = 0;
 
@@ -230,10 +231,9 @@ public class DLockToken {
   // -------------------------------------------------------------------------
 
   /**
-   * Destroys this lock token. Caller must synchronize on this lock token.
+   * Destroys this lock token.
    */
   synchronized void destroy() {
-    // checkDestroyed();
     this.destroyed = true;
   }
 
@@ -302,14 +302,14 @@ public class DLockToken {
    * @param remoteThread identity of the leasing thread
    * @return true if lease for this lock token is successfully granted
    */
-  synchronized boolean grantLock(long newLeaseExpireTime, int newLeaseId, int newRecursion,
+  synchronized void grantLock(long newLeaseExpireTime, int newLeaseId, int newRecursion,
       RemoteThread remoteThread) {
 
     Assert.assertTrue(remoteThread != null);
     Assert.assertTrue(newLeaseId > -1, "Invalid attempt to grant lock with leaseId " + newLeaseId);
 
     checkDestroyed();
-    checkForExpiration();
+    checkForExpiration(); // TODO: this should throw.
 
     this.ignoreForRecovery = false;
     this.leaseExpireTime = newLeaseExpireTime;
@@ -321,8 +321,6 @@ public class DLockToken {
     if (logger.isTraceEnabled(LogMarker.DLS)) {
       logger.trace(LogMarker.DLS, "[DLockToken.grantLock.client] granted {}", this);
     }
-
-    return true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/geode/blob/190cfed8/geode-core/src/test/java/org/apache/geode/distributed/internal/locks/DLockServiceCharacterizationTests.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/locks/DLockServiceCharacterizationTests.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/locks/DLockServiceCharacterizationTests.java
new file mode 100644
index 0000000..ba300c4
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/locks/DLockServiceCharacterizationTests.java
@@ -0,0 +1,124 @@
+/*
+ * 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.geode.distributed.internal.locks;
+
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.awaitility.Awaitility.await;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.ExpirationAction;
+import org.apache.geode.cache.ExpirationAttributes;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.distributed.DistributedLockService;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+public class DLockServiceCharacterizationTests {
+  private Cache cache;
+  private DistributedRegion testRegion;
+  private DistributedLockService dLockService;
+
+  @Before
+  public void setUp() {
+    Properties properties = new Properties();
+    properties.setProperty(MCAST_PORT, "0");
+
+    cache = new CacheFactory(properties).create();
+    testRegion = (DistributedRegion) cache.createRegionFactory(RegionShortcut.REPLICATE)
+        .setScope(Scope.GLOBAL)
+        .setEntryTimeToLive(new ExpirationAttributes(1, ExpirationAction.DESTROY))
+        .create("testRegion");
+    testRegion.becomeLockGrantor();
+
+    dLockService = DLockService.create("testService", cache.getDistributedSystem());
+  }
+
+  @After
+  public void tearDown() {
+    cache.close();
+  }
+
+  @Test
+  public void reentrantLockIncreasesReentrancy() {
+    assertTrue(dLockService.lock("key1", -1, -1));
+    DLockToken key1 = ((DLockService) dLockService).getToken("key1");
+
+    assertEquals(0, key1.getRecursion());
+    assertEquals(1, key1.getUsageCount());
+    // reentrancy + 1
+    assertTrue(dLockService.lock("key1", -1, -1));
+
+    assertEquals(1, key1.getRecursion());
+    assertEquals(2, key1.getUsageCount());
+
+    dLockService.unlock("key1");
+    assertEquals(0, key1.getRecursion());
+    assertEquals(1, key1.getUsageCount());
+
+    dLockService.unlock("key1");
+    assertTokenIsUnused(key1);
+  }
+
+  @Test
+  public void threadWaitingOnLockIncreasesUsageCount() {
+    assertTrue(dLockService.lock("key1", -1, -1));
+    DLockToken key1 = ((DLockService) dLockService).getToken("key1");
+
+    assertEquals(0, key1.getRecursion());
+    assertEquals(1, key1.getUsageCount());
+    assertEquals(Thread.currentThread(), key1.getThread());
+
+    Thread otherThread = new Thread(() -> dLockService.lock("key1", -1, -1));
+    otherThread.start();
+
+    // otherThread should be waiting for lock.
+
+    await("other thread is waiting on this lock").atMost(3, TimeUnit.SECONDS)
+        .until(() -> key1.getUsageCount() == 2);
+    assertEquals(0, key1.getRecursion());
+    assertEquals(Thread.currentThread(), key1.getThread());
+
+    dLockService.unlock("key1");
+
+    await("other thread has acquired this lock").atMost(3, TimeUnit.SECONDS)
+        .until(() -> key1.getThread() == otherThread);
+
+    assertEquals(0, key1.getRecursion());
+    assertEquals(1, key1.getUsageCount());
+
+    // We can unlock from a different thread than locked it.
+    dLockService.unlock("key1");
+
+    assertTokenIsUnused(key1);
+  }
+
+  private void assertTokenIsUnused(DLockToken dLockToken) {
+    assertEquals(0, dLockToken.getRecursion());
+    assertEquals(0, dLockToken.getUsageCount());
+    assertEquals(null, dLockToken.getThread());
+    assertEquals(null, dLockToken.getLesseeThread());
+    assertEquals(-1, dLockToken.getLeaseId());
+  }
+}