You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2015/12/17 23:20:56 UTC

incubator-geode git commit: GEODE-678: dlock reenter will no longer timeout

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 0e366b026 -> d31a85ff7


GEODE-678: dlock reenter will no longer timeout


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

Branch: refs/heads/develop
Commit: d31a85ff7898732857d5fd1abee8030c8beeca66
Parents: 0e366b0
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Dec 15 15:36:24 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 17 14:18:23 2015 -0800

----------------------------------------------------------------------
 .../internal/locks/DLockGrantor.java            |  5 +-
 .../locks/DLockReentrantLockJUnitTest.java      | 84 ++++++++++++++++++++
 2 files changed, 85 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d31a85ff/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
index b4e646c..a5b8e0f 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
@@ -984,10 +984,7 @@ public class DLockGrantor {
     }
     try {
       checkDestroyed();
-      if (request.checkForTimeout()) {
-        // no cleanup here because we bypassed lock permissions
-        return 0;
-      }
+      // to fix GEODE-678 no longer call request.checkForTimeout
       DLockGrantToken grant = getGrantToken(request.getObjectName());
       if (grant == null) {
         if (logger.isTraceEnabled(LogMarker.DLS)) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d31a85ff/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java
new file mode 100644
index 0000000..2dbd231
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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 com.gemstone.gemfire.distributed.internal.locks;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+
+@Category(IntegrationTest.class)
+public class DLockReentrantLockJUnitTest {
+  final long id = 1213L;
+
+  private Cache cache;
+  private Region<Long, String> region;
+
+  @Before
+  public void setup() {
+    cache = new CacheFactory().set("mcast-port", "0").set("locators", "").create();
+    final RegionFactory<Long, String> regionFactory = cache.createRegionFactory("REPLICATE");
+    regionFactory.setScope(Scope.GLOBAL);
+    region = regionFactory.create("ReentrantLockRegion");
+    region.put(id, new String("TestValue1"));
+  }
+
+  @After
+  public void tearDown() {
+    cache.close();
+  }
+
+  /**
+   * Tests GEM-96/GEODE-678
+   */
+  @Test
+  public void testReentrantLock() throws Exception {
+
+    Assert.assertEquals(Scope.GLOBAL, region.getAttributes().getScope());
+
+    final Lock lock1 = region.getDistributedLock(id);
+    final Lock lock2 = region.getDistributedLock(id);
+
+    for (int i = 0; i < 50; i++) {
+      lock1.lock();
+      boolean reenteredLock = false;
+      try {
+        reenteredLock = lock2.tryLock(1, TimeUnit.NANOSECONDS);
+        if (!reenteredLock) {
+          System.out.println("ERROR: could not reenter lock");
+        }
+        Assert.assertTrue("Failed getting lock at 2:" + i, reenteredLock);
+      } finally {
+        if (reenteredLock) {
+          lock2.unlock();
+        }
+        lock1.unlock();
+      }
+    }
+  }
+}