You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by ch...@apache.org on 2015/10/15 13:59:29 UTC

incubator-reef git commit: [REEF-588] DeadlockInfo throws NullPointerException when deadlock is not detected

Repository: incubator-reef
Updated Branches:
  refs/heads/master a9fc25862 -> ed66b6e99


[REEF-588] DeadlockInfo throws NullPointerException when deadlock is not detected

This patch:

  * Adds a DeadlockInfoTestWithDeadlockAbsent to reproduce the NPE and test the normal behavior when a deadlock does not exist
  * Renames the DeadlockInfoTest to DeadlockInfoTestWithDeadlockPresent to avoid confusion
  * Adds a check to the DeadLockInfo constructor to test if the result of findDeadlockedThreads() is null
  * Makes DeadlockInfo return an empty ThreadInfo array if no deadlocks are found
  * Makes ThreadLogger log "none" if no deadlocks are found

JIRA:
  [REEF-588](https://issues.apache.org/jira/browse/REEF-588)

Pull Request:
  Closes #564


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

Branch: refs/heads/master
Commit: ed66b6e991df55036a97f02ff52ed13cc71f4565
Parents: a9fc258
Author: sergey.dudoladov@tu-berlin.de <se...@tu-berlin.de>
Authored: Thu Sep 24 19:29:14 2015 +0200
Committer: Brian Cho <ch...@apache.org>
Committed: Thu Oct 15 20:58:34 2015 +0900

----------------------------------------------------------------------
 .../java/org/apache/reef/util/DeadlockInfo.java |  11 +-
 .../java/org/apache/reef/util/ThreadLogger.java |  10 +-
 .../org/apache/reef/util/DeadlockInfoTest.java  | 176 -------------------
 .../DeadlockInfoWithDeadlockAbsentTest.java     | 143 +++++++++++++++
 .../DeadlockInfoWithDeadlockPresentTest.java    | 176 +++++++++++++++++++
 5 files changed, 337 insertions(+), 179 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/ed66b6e9/lang/java/reef-common/src/main/java/org/apache/reef/util/DeadlockInfo.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/util/DeadlockInfo.java b/lang/java/reef-common/src/main/java/org/apache/reef/util/DeadlockInfo.java
index 35fb591..a5f49a8 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/util/DeadlockInfo.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/util/DeadlockInfo.java
@@ -36,11 +36,18 @@ import java.util.Map;
 final class DeadlockInfo {
   private final ThreadMXBean mxBean;
   private final ThreadInfo[] deadlockedThreads;
+  private final long[] deadlockedThreadsIDs;
+  private static final ThreadInfo[] EMPTY_ARRAY = new ThreadInfo[0];
   private final Map<ThreadInfo, Map<StackTraceElement, List<MonitorInfo>>> monitorLockedElements;
 
   public DeadlockInfo() {
     mxBean = ManagementFactory.getThreadMXBean();
-    deadlockedThreads = mxBean.getThreadInfo(mxBean.findDeadlockedThreads(), true, true);
+    deadlockedThreadsIDs = mxBean.findDeadlockedThreads();
+
+    deadlockedThreads = (null == deadlockedThreadsIDs)
+                        ? EMPTY_ARRAY
+                        : mxBean.getThreadInfo(deadlockedThreadsIDs, true, true);
+
     monitorLockedElements = new HashMap<>();
     for (final ThreadInfo threadInfo : deadlockedThreads) {
       monitorLockedElements.put(threadInfo, constructMonitorLockedElements(threadInfo));
@@ -48,7 +55,7 @@ final class DeadlockInfo {
   }
 
   /**
-   * @return An array of deadlocked threads
+   * @return A (potentially empty) array of deadlocked threads
    */
   public ThreadInfo[] getDeadlockedThreads() {
     return deadlockedThreads;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/ed66b6e9/lang/java/reef-common/src/main/java/org/apache/reef/util/ThreadLogger.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/util/ThreadLogger.java b/lang/java/reef-common/src/main/java/org/apache/reef/util/ThreadLogger.java
index 210663d..104b468 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/util/ThreadLogger.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/util/ThreadLogger.java
@@ -98,7 +98,15 @@ public final class ThreadLogger {
     final StringBuilder message = new StringBuilder(prefix);
 
     final DeadlockInfo deadlockInfo = new DeadlockInfo();
-    for (final ThreadInfo threadInfo : deadlockInfo.getDeadlockedThreads()) {
+
+    final ThreadInfo[] deadlockedThreads = deadlockInfo.getDeadlockedThreads();
+
+    if (0 == deadlockedThreads.length) {
+      message.append(" none ");
+      return message.toString();
+    }
+
+    for (final ThreadInfo threadInfo : deadlockedThreads) {
       message.append(threadPrefix).append("Thread '").append(threadInfo.getThreadName())
           .append("' with state ").append(threadInfo.getThreadState());
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/ed66b6e9/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoTest.java b/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoTest.java
deleted file mode 100644
index a922800..0000000
--- a/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoTest.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * 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.reef.util;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.lang.management.ThreadInfo;
-
-import java.util.concurrent.BrokenBarrierException;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Test DeadlockInfo by creating a deadlock.
- */
-public final class DeadlockInfoTest {
-
-  private static final Logger LOG = Logger.getLogger(DeadlockInfoTest.class.getName());
-
-  private static final long TIMEOUT_MILLIS = 100;
-
-  private static final Lock LOCK_1 = new ReentrantLock();
-  private static final Lock LOCK_2 = new ReentrantLock();
-
-  private static Thread thread1;
-  private static Thread thread2;
-
-  /**
-   * Create a deadlock consisting of two threads.
-   *
-   * setUpClass sleeps for TIMEOUT_MILLIS to allow the threads to progress into deadlock.
-   */
-  @BeforeClass
-  public static void setUpClass() {
-    createDeadlock();
-    threadSleep(TIMEOUT_MILLIS);
-  }
-
-  /**
-   * Remove  the deadlock by interrupting the first thread.
-   *
-   * This ensures that future DeadlockInfo test does not
-   * detect the unnecessary deadlock.
-   */
-  @AfterClass
-  public static void tearDownClass() {
-    thread1.interrupt();
-    threadSleep(TIMEOUT_MILLIS);
-  }
-
-  /**
-   * The first thread holds the LOCK_1, and waits interruptibly on the LOCK_2;
-   * the other one holds the LOCK_2 and waits on the LOCK_1.
-   *
-   * The barrier in between lock acquisition ensures that the deadlock occurs.
-   *
-   * Since there is no way to kill a thread, to resolve the deadlock we instead
-   * interrupt the first thread and simply allow it to finish. This releases LOCK_1.
-   *
-   * The second thread should then terminate normally.
-   */
-  private static void createDeadlock() {
-
-    final CyclicBarrier barrier = new CyclicBarrier(2);
-
-    thread1 = new Thread() {
-          @Override
-          public void run() {
-
-            try {
-              LOCK_1.lock();
-              barrierAwait(barrier);
-              LOCK_2.lockInterruptibly();
-            } catch (InterruptedException e) {
-              LOG.info(Thread.currentThread().getName() + " is interrupted."
-                      + " This interrupt is expected because it resolves the deadlock.");
-            }
-
-          }
-      };
-
-    thread2 = new Thread() {
-          @Override
-          public void run() {
-            LOCK_2.lock();
-            barrierAwait(barrier);
-            LOCK_1.lock();
-          }
-      };
-
-    thread1.start();
-    thread2.start();
-
-  }
-
-  /**
-   * Test that DeadlockInfo returns the expected values given the deadlock.
-   */
-  @Test
-  public void testDeadlockInfo() {
-
-    final DeadlockInfo deadlockInfo = new DeadlockInfo();
-
-    final ThreadInfo[] threadInfos = deadlockInfo.getDeadlockedThreads();
-    assertEquals("There must be two deadlocked threads", 2, threadInfos.length);
-
-    for (final ThreadInfo threadInfo : deadlockInfo.getDeadlockedThreads()) {
-      final String waitingLockString = deadlockInfo.getWaitingLockString(threadInfo);
-      assertNotNull("Each thread should wait on a lock and"
-              + " hence have the non-null waitingLockString", waitingLockString);
-      assertTrue("Each Thread should wait on the ReentrantLock", waitingLockString.contains("ReentrantLock"));
-    }
-
-  }
-
-  @Test
-  public void testLogDeadlockInfo() {
-    LOG.log(Level.INFO, ThreadLogger.getFormattedDeadlockInfo("Deadlock test, this deadlock is expected"));
-  }
-
-  /**
-   * Once the barrier is met, the threads proceed to deadlock.
-   */
-  private static void barrierAwait(final CyclicBarrier barrier) {
-    try {
-      barrier.await(TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
-    } catch (final InterruptedException e) {
-      e.printStackTrace();
-      fail("Unexpected exception");
-    } catch (final BrokenBarrierException e) {
-      e.printStackTrace();
-      fail("Unexpected exception");
-    } catch (final TimeoutException e) {
-      e.printStackTrace();
-      fail("Unexpected exception");
-    }
-  }
-
-  private static void threadSleep(final long millis) {
-    try {
-      Thread.sleep(millis);
-    } catch (final InterruptedException e) {
-      e.printStackTrace();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/ed66b6e9/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoWithDeadlockAbsentTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoWithDeadlockAbsentTest.java b/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoWithDeadlockAbsentTest.java
new file mode 100644
index 0000000..307f38c
--- /dev/null
+++ b/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoWithDeadlockAbsentTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.reef.util;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Test DeadlockInfo by requesting information about
+ * non-deadlocked threads.
+ *
+ * Reproduces REEF-588.
+ */
+public class DeadlockInfoWithDeadlockAbsentTest {
+
+  private static final Logger LOG = Logger.getLogger(DeadlockInfoWithDeadlockAbsentTest.class.getName());
+
+  private static final long TIMEOUT_MILLIS = 100;
+
+  private static final CountDownLatch FIRST_LATCH = new CountDownLatch(1);
+  private static final CountDownLatch SECOND_LATCH = new CountDownLatch(1);
+
+  /**
+    * Create a situation where a deadlock is possible but not present.
+    *
+    * Sleep for TIMEOUT_MILLIS to allow this situation to set up.
+   *
+  */
+  @BeforeClass
+  public static void setUp() {
+    startNonDeadlockedThreads();
+    threadSleep(TIMEOUT_MILLIS);
+  }
+
+  /**
+   * Test the normal DeadlockInfo behaviour in the
+   * absence of deadlocks.
+   *
+   * DeadlockInfo instantiation reproduces the REEF-588
+   * because the NPE was thrown in the DeadlockInfo constructor.
+   */
+  @Test
+  public void testDeadlockInfoWithDeadlockAbsent()
+          throws NullPointerException {
+
+    final DeadlockInfo deadlockInfo = new DeadlockInfo();
+    LOG.log(Level.INFO, ThreadLogger.getFormattedDeadlockInfo(
+              "DeadlockInfo test, none deadlocks expected. Deadlocks found: "));
+    Assert.assertEquals("DeadlockInfo found deadlocks when none should exist.", 0,
+              deadlockInfo.getDeadlockedThreads().length);
+
+  }
+
+  /**
+   * Test logging in the absence of deadlocks.
+   */
+  @Test
+  public void testLogDeadlockInfo() throws NullPointerException {
+    LOG.log(Level.INFO, ThreadLogger.getFormattedDeadlockInfo(
+            "DeadlockInfo test, none deadlocks expected. Deadlocks found: "));
+  }
+
+  /**
+   * Create a situation where a deadlock is possible but not present.
+   *
+   * Assume there are two resources guarded by latches.
+   * The deadlock is possible if multiple threads attempt to acquire both latches.
+   *
+   * Spawn two threads so that each thread acquires only one latch.
+   * By design circular wait between the threads can't occur, so the deadlock can't exist.
+   *
+   * The threads wait on their latches until the DeadlockInfo tests finish.
+   * Then the tearDown() wakes up the threads to avoid liveness issues.
+   */
+  private static void startNonDeadlockedThreads() {
+
+    final Thread thread1 = new Thread(){
+        @Override
+        public void run(){
+            awaitOnLatch(FIRST_LATCH);
+        }
+    };
+
+    final Thread thread2 = new Thread(){
+        @Override
+        public void run(){
+            awaitOnLatch(SECOND_LATCH);
+        }
+    };
+
+    thread1.start();
+    thread2.start();
+  }
+
+  /**
+   * Allow both threads to finish.
+   */
+  @AfterClass
+  public static void tearDown() {
+    FIRST_LATCH.countDown();
+    SECOND_LATCH.countDown();
+  }
+
+  private static void awaitOnLatch(final CountDownLatch latch) {
+    try {
+      latch.await();
+    } catch (final InterruptedException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private static void threadSleep(final long millis) {
+    try {
+      Thread.sleep(millis);
+    } catch (final InterruptedException e) {
+      e.printStackTrace();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/ed66b6e9/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoWithDeadlockPresentTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoWithDeadlockPresentTest.java b/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoWithDeadlockPresentTest.java
new file mode 100644
index 0000000..c211ff3
--- /dev/null
+++ b/lang/java/reef-common/src/test/java/org/apache/reef/util/DeadlockInfoWithDeadlockPresentTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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.reef.util;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.lang.management.ThreadInfo;
+
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test DeadlockInfo by creating a deadlock.
+ */
+
+public final class DeadlockInfoWithDeadlockPresentTest {
+  private static final Logger LOG = Logger.getLogger(DeadlockInfoWithDeadlockPresentTest.class.getName());
+
+  private static final long TIMEOUT_MILLIS = 100;
+
+  private static final Lock LOCK_1 = new ReentrantLock();
+  private static final Lock LOCK_2 = new ReentrantLock();
+
+  private static Thread thread1;
+  private static Thread thread2;
+
+  /**
+   * Create a deadlock consisting of two threads.
+   *
+   * setUpClass sleeps for TIMEOUT_MILLIS to allow the threads to progress into deadlock.
+   */
+  @BeforeClass
+  public static void setUpClass() {
+    createDeadlock();
+    threadSleep(TIMEOUT_MILLIS);
+  }
+
+  /**
+   * Remove  the deadlock by interrupting the first thread.
+   *
+   * This ensures that future DeadlockInfo test does not
+   * detect the unnecessary deadlock.
+   */
+  @AfterClass
+  public static void tearDownClass() {
+    thread1.interrupt();
+    threadSleep(TIMEOUT_MILLIS);
+  }
+
+  /**
+   * The first thread holds the LOCK_1, and waits interruptibly on the LOCK_2;
+   * the other one holds the LOCK_2 and waits on the LOCK_1.
+   *
+   * The barrier in between lock acquisition ensures that the deadlock occurs.
+   *
+   * Since there is no way to kill a thread, to resolve the deadlock we instead
+   * interrupt the first thread and simply allow it to finish. This releases LOCK_1.
+   *
+   * The second thread should then terminate normally.
+   */
+  private static void createDeadlock() {
+
+    final CyclicBarrier barrier = new CyclicBarrier(2);
+
+    thread1 = new Thread() {
+          @Override
+          public void run() {
+
+            try {
+              LOCK_1.lock();
+              barrierAwait(barrier);
+              LOCK_2.lockInterruptibly();
+            } catch (InterruptedException e) {
+              LOG.info(Thread.currentThread().getName() + " is interrupted."
+                      + " This interrupt is expected because it resolves the deadlock.");
+            }
+
+          }
+      };
+
+    thread2 = new Thread() {
+          @Override
+          public void run() {
+            LOCK_2.lock();
+            barrierAwait(barrier);
+            LOCK_1.lock();
+          }
+      };
+
+    thread1.start();
+    thread2.start();
+
+  }
+
+  /**
+   * Test that DeadlockInfo returns the expected values given the deadlock.
+   */
+  @Test
+  public void testDeadlockInfo() {
+
+    final DeadlockInfo deadlockInfo = new DeadlockInfo();
+
+    final ThreadInfo[] threadInfos = deadlockInfo.getDeadlockedThreads();
+    assertEquals("There must be two deadlocked threads", 2, threadInfos.length);
+
+    for (final ThreadInfo threadInfo : deadlockInfo.getDeadlockedThreads()) {
+      final String waitingLockString = deadlockInfo.getWaitingLockString(threadInfo);
+      assertNotNull("Each thread should wait on a lock and"
+              + " hence have the non-null waitingLockString", waitingLockString);
+      assertTrue("Each Thread should wait on the ReentrantLock", waitingLockString.contains("ReentrantLock"));
+    }
+
+  }
+
+  @Test
+  public void testLogDeadlockInfo() {
+    LOG.log(Level.INFO, ThreadLogger.getFormattedDeadlockInfo("Deadlock test, this deadlock is expected"));
+  }
+
+  /**
+   * Once the barrier is met, the threads proceed to deadlock.
+   */
+  private static void barrierAwait(final CyclicBarrier barrier) {
+    try {
+      barrier.await(TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+    } catch (final InterruptedException e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    } catch (final BrokenBarrierException e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    } catch (final TimeoutException e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    }
+  }
+
+  private static void threadSleep(final long millis) {
+    try {
+      Thread.sleep(millis);
+    } catch (final InterruptedException e) {
+      e.printStackTrace();
+    }
+  }
+
+}