You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@distributedlog.apache.org by si...@apache.org on 2017/01/05 00:09:04 UTC

incubator-distributedlog git commit: DL-165: Add TestTimedOutTestsListener to dump timed out cases thread dump

Repository: incubator-distributedlog
Updated Branches:
  refs/heads/master 1a30b0ceb -> ea3c1143f


DL-165: Add TestTimedOutTestsListener to dump timed out cases thread dump

Author: xieliang <xi...@gmail.com>

Reviewers: Leigh Stewart <ls...@apache.org>

Closes #91 from xieliang/DL-165-TimedOutTestsListener


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

Branch: refs/heads/master
Commit: ea3c1143f9e2718d0d86e8b1c8f3a7e51ac19c4d
Parents: 1a30b0c
Author: xieliang <xi...@gmail.com>
Authored: Wed Jan 4 16:09:01 2017 -0800
Committer: Sijie Guo <si...@twitter.com>
Committed: Wed Jan 4 16:09:01 2017 -0800

----------------------------------------------------------------------
 NOTICE                                          |   9 +
 distributedlog-client/pom.xml                   |  15 +-
 distributedlog-core/pom.xml                     |  15 +-
 distributedlog-protocol/pom.xml                 |  12 ++
 .../TestTimedOutTestsListener.java              | 183 +++++++++++++++++++
 .../distributedlog/TimedOutTestsListener.java   | 168 +++++++++++++++++
 distributedlog-service/pom.xml                  |  24 +++
 7 files changed, 424 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/ea3c1143/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 5245e00..7281f12 100644
--- a/NOTICE
+++ b/NOTICE
@@ -10,3 +10,12 @@ The Apache Software Foundation (http://www.apache.org/).
 
 Portions of this software were developed by Twitter.
 Copyright Twitter, 2016
+
+This product contains a modified portion of 'Apache Hadoop', a Java framework that
+allows for the distributed processing of large data sets across clusters of computers
+using simple programming models., which can be obtained at:
+
+  * LICENSE:
+    * LICENSE (Apache License 2.0)
+  * HOMEPAGE:
+    * https://hadoop.apache.org/

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/ea3c1143/distributedlog-client/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-client/pom.xml b/distributedlog-client/pom.xml
index 1e88672..aad5093 100644
--- a/distributedlog-client/pom.xml
+++ b/distributedlog-client/pom.xml
@@ -85,6 +85,13 @@
       <version>1.9.5</version>
       <scope>test</scope>
     </dependency> 
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>distributedlog-protocol</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <build>
     <plugins>
@@ -121,12 +128,18 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
-        <version>2.9</version>
+        <version>2.19.1</version>
         <configuration>
           <redirectTestOutputToFile>true</redirectTestOutputToFile>
           <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G</argLine>
           <forkMode>always</forkMode>
           <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
+          <properties>
+            <property>
+              <name>listener</name>
+              <value>com.twitter.distributedlog.TimedOutTestsListener</value>
+            </property>
+          </properties>
         </configuration>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/ea3c1143/distributedlog-core/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-core/pom.xml b/distributedlog-core/pom.xml
index 4da1849..c5329aa 100644
--- a/distributedlog-core/pom.xml
+++ b/distributedlog-core/pom.xml
@@ -145,6 +145,13 @@
       <version>1.9.5</version>
       <scope>test</scope>
     </dependency> 
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>distributedlog-protocol</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <build>
     <plugins>
@@ -190,12 +197,18 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
-        <version>2.9</version>
+        <version>2.19.1</version>
         <configuration>
           <redirectTestOutputToFile>true</redirectTestOutputToFile>
           <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G</argLine>
           <forkMode>always</forkMode>
           <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
+          <properties>
+            <property>
+              <name>listener</name>
+              <value>com.twitter.distributedlog.TimedOutTestsListener</value>
+            </property>
+          </properties>
         </configuration>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/ea3c1143/distributedlog-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/pom.xml b/distributedlog-protocol/pom.xml
index 3adae05..5e55482 100644
--- a/distributedlog-protocol/pom.xml
+++ b/distributedlog-protocol/pom.xml
@@ -107,6 +107,18 @@
         </executions>
       </plugin>
       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.2</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>
         <configuration>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/ea3c1143/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestTimedOutTestsListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestTimedOutTestsListener.java b/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestTimedOutTestsListener.java
new file mode 100644
index 0000000..8d200ec
--- /dev/null
+++ b/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestTimedOutTestsListener.java
@@ -0,0 +1,183 @@
+/**
+ * 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.twitter.distributedlog;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.notification.Failure;
+
+/**
+ * Test Case for {@link TimedOutTestsListener}.
+ */
+public class TestTimedOutTestsListener {
+
+    private static class Deadlock {
+        private CyclicBarrier barrier = new CyclicBarrier(6);
+
+        public Deadlock() {
+            DeadlockThread[] dThreads = new DeadlockThread[6];
+
+            Monitor a = new Monitor("a");
+            Monitor b = new Monitor("b");
+            Monitor c = new Monitor("c");
+            dThreads[0] = new DeadlockThread("MThread-1", a, b);
+            dThreads[1] = new DeadlockThread("MThread-2", b, c);
+            dThreads[2] = new DeadlockThread("MThread-3", c, a);
+
+            Lock d = new ReentrantLock();
+            Lock e = new ReentrantLock();
+            Lock f = new ReentrantLock();
+
+            dThreads[3] = new DeadlockThread("SThread-4", d, e);
+            dThreads[4] = new DeadlockThread("SThread-5", e, f);
+            dThreads[5] = new DeadlockThread("SThread-6", f, d);
+
+            // make them daemon threads so that the test will exit
+            for (int i = 0; i < 6; i++) {
+                dThreads[i].setDaemon(true);
+                dThreads[i].start();
+            }
+        }
+
+        class DeadlockThread extends Thread {
+            private Lock lock1 = null;
+
+            private Lock lock2 = null;
+
+            private Monitor mon1 = null;
+
+            private Monitor mon2 = null;
+
+            private boolean useSync;
+
+            DeadlockThread(String name, Lock lock1, Lock lock2) {
+                super(name);
+                this.lock1 = lock1;
+                this.lock2 = lock2;
+                this.useSync = true;
+            }
+
+            DeadlockThread(String name, Monitor mon1, Monitor mon2) {
+                super(name);
+                this.mon1 = mon1;
+                this.mon2 = mon2;
+                this.useSync = false;
+            }
+
+            public void run() {
+                if (useSync) {
+                    syncLock();
+                } else {
+                    monitorLock();
+                }
+            }
+
+            private void syncLock() {
+                lock1.lock();
+                try {
+                    try {
+                        barrier.await();
+                    } catch (Exception e) {
+                    }
+                    goSyncDeadlock();
+                } finally {
+                    lock1.unlock();
+                }
+            }
+
+            private void goSyncDeadlock() {
+                try {
+                    barrier.await();
+                } catch (Exception e) {
+                }
+                lock2.lock();
+                throw new RuntimeException("should not reach here.");
+            }
+
+            private void monitorLock() {
+                synchronized (mon1) {
+                    try {
+                        barrier.await();
+                    } catch (Exception e) {
+                    }
+                    goMonitorDeadlock();
+                }
+            }
+
+            private void goMonitorDeadlock() {
+                try {
+                    barrier.await();
+                } catch (Exception e) {
+                }
+                synchronized (mon2) {
+                    throw new RuntimeException(getName() + " should not reach here.");
+                }
+            }
+        }
+
+        class Monitor {
+            String name;
+
+            Monitor(String name) {
+                this.name = name;
+            }
+        }
+
+    }
+
+    @Test(timeout = 500)
+    public void testThreadDumpAndDeadlocks() throws Exception {
+        new Deadlock();
+        String s = null;
+        while (true) {
+            s = TimedOutTestsListener.buildDeadlockInfo();
+            if (s != null) {
+                break;
+            }
+            Thread.sleep(100);
+        }
+
+        Assert.assertEquals(3, countStringOccurrences(s, "BLOCKED"));
+
+        Failure failure = new Failure(null, new Exception(TimedOutTestsListener.TEST_TIMED_OUT_PREFIX));
+        StringWriter writer = new StringWriter();
+        new TimedOutTestsListener(new PrintWriter(writer)).testFailure(failure);
+        String out = writer.toString();
+
+        Assert.assertTrue(out.contains("THREAD DUMP"));
+        Assert.assertTrue(out.contains("DEADLOCKS DETECTED"));
+
+        System.out.println(out);
+    }
+
+    private int countStringOccurrences(String s, String substr) {
+        int n = 0;
+        int index = 0;
+        while ((index = s.indexOf(substr, index) + 1) != 0) {
+            n++;
+        }
+        return n;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/ea3c1143/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TimedOutTestsListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TimedOutTestsListener.java b/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TimedOutTestsListener.java
new file mode 100644
index 0000000..f2e80d2
--- /dev/null
+++ b/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TimedOutTestsListener.java
@@ -0,0 +1,168 @@
+/**
+ * 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.twitter.distributedlog;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.management.LockInfo;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MonitorInfo;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.junit.runner.notification.Failure;
+import org.junit.runner.notification.RunListener;
+
+/**
+ * JUnit run listener which prints full thread dump into System.err in case a test is failed due to
+ * timeout.
+ */
+public class TimedOutTestsListener extends RunListener {
+
+    static final String TEST_TIMED_OUT_PREFIX = "test timed out after";
+
+    private static String indent = "    ";
+
+    private final PrintWriter output;
+
+    public TimedOutTestsListener() {
+        this.output = new PrintWriter(System.err);
+    }
+
+    public TimedOutTestsListener(PrintWriter output) {
+        this.output = output;
+    }
+
+    @Override
+    public void testFailure(Failure failure) throws Exception {
+        if (failure != null && failure.getMessage() != null && failure.getMessage().startsWith(TEST_TIMED_OUT_PREFIX)) {
+            output.println("====> TEST TIMED OUT. PRINTING THREAD DUMP. <====");
+            output.println();
+            output.print(buildThreadDiagnosticString());
+        }
+    }
+
+    public static String buildThreadDiagnosticString() {
+        StringWriter sw = new StringWriter();
+        PrintWriter output = new PrintWriter(sw);
+
+        DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd hh:mm:ss,SSS");
+        output.println(String.format("Timestamp: %s", dateFormat.format(new Date())));
+        output.println();
+        output.println(buildThreadDump());
+
+        String deadlocksInfo = buildDeadlockInfo();
+        if (deadlocksInfo != null) {
+            output.println("====> DEADLOCKS DETECTED <====");
+            output.println();
+            output.println(deadlocksInfo);
+        }
+
+        return sw.toString();
+    }
+
+    static String buildThreadDump() {
+        StringBuilder dump = new StringBuilder();
+        Map<Thread, StackTraceElement[]> stackTraces = Thread.getAllStackTraces();
+        for (Map.Entry<Thread, StackTraceElement[]> e : stackTraces.entrySet()) {
+            Thread thread = e.getKey();
+            dump.append(String.format("\"%s\" %s prio=%d tid=%d %s\njava.lang.Thread.State: %s", thread.getName(),
+                (thread.isDaemon() ? "daemon" : ""), thread.getPriority(), thread.getId(),
+                Thread.State.WAITING.equals(thread.getState()) ? "in Object.wait()"
+                        : StringUtils.lowerCase(thread.getState().name()),
+                Thread.State.WAITING.equals(thread.getState()) ? "WAITING (on object monitor)" : thread.getState()));
+            for (StackTraceElement stackTraceElement : e.getValue()) {
+                dump.append("\n        at ");
+                dump.append(stackTraceElement);
+            }
+            dump.append("\n");
+        }
+        return dump.toString();
+    }
+
+    static String buildDeadlockInfo() {
+        ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+        long[] threadIds = threadBean.findMonitorDeadlockedThreads();
+        if (threadIds != null && threadIds.length > 0) {
+            StringWriter stringWriter = new StringWriter();
+            PrintWriter out = new PrintWriter(stringWriter);
+
+            ThreadInfo[] infos = threadBean.getThreadInfo(threadIds, true, true);
+            for (ThreadInfo ti : infos) {
+                printThreadInfo(ti, out);
+                printLockInfo(ti.getLockedSynchronizers(), out);
+                out.println();
+            }
+
+            out.close();
+            return stringWriter.toString();
+        } else {
+            return null;
+        }
+    }
+
+    private static void printThreadInfo(ThreadInfo ti, PrintWriter out) {
+        // print thread information
+        printThread(ti, out);
+
+        // print stack trace with locks
+        StackTraceElement[] stacktrace = ti.getStackTrace();
+        MonitorInfo[] monitors = ti.getLockedMonitors();
+        for (int i = 0; i < stacktrace.length; i++) {
+            StackTraceElement ste = stacktrace[i];
+            out.println(indent + "at " + ste.toString());
+            for (MonitorInfo mi : monitors) {
+                if (mi.getLockedStackDepth() == i) {
+                    out.println(indent + "  - locked " + mi);
+                }
+            }
+        }
+        out.println();
+    }
+
+    private static void printThread(ThreadInfo ti, PrintWriter out) {
+        out.print("\"" + ti.getThreadName() + "\"" + " Id=" + ti.getThreadId() + " in " + ti.getThreadState());
+        if (ti.getLockName() != null) {
+            out.print(" on lock=" + ti.getLockName());
+        }
+        if (ti.isSuspended()) {
+            out.print(" (suspended)");
+        }
+        if (ti.isInNative()) {
+            out.print(" (running in native)");
+        }
+        out.println();
+        if (ti.getLockOwnerName() != null) {
+            out.println(indent + " owned by " + ti.getLockOwnerName() + " Id=" + ti.getLockOwnerId());
+        }
+    }
+
+    private static void printLockInfo(LockInfo[] locks, PrintWriter out) {
+        out.println(indent + "Locked synchronizers: count = " + locks.length);
+        for (LockInfo li : locks) {
+            out.println(indent + "  - " + li);
+        }
+        out.println();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/ea3c1143/distributedlog-service/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-service/pom.xml b/distributedlog-service/pom.xml
index 154dedb..8399675 100644
--- a/distributedlog-service/pom.xml
+++ b/distributedlog-service/pom.xml
@@ -139,6 +139,13 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>distributedlog-protocol</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <build>
     <plugins>
@@ -177,6 +184,23 @@
         </executions>
       </plugin>
       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.19.1</version>
+        <configuration>
+          <redirectTestOutputToFile>true</redirectTestOutputToFile>
+          <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G</argLine>
+          <forkMode>always</forkMode>
+          <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
+          <properties>
+            <property>
+              <name>listener</name>
+              <value>com.twitter.distributedlog.TimedOutTestsListener</value>
+            </property>
+          </properties>
+        </configuration>
+      </plugin>
+      <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>
         <configuration>