Forráskód Böngészése

ZOOKEEPER-597. ASyncHammerTest is failing intermittently on hudson trunk (Patrick Hunt via mahadev)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@884229 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 15 éve
szülő
commit
170dd6bb59

+ 3 - 0
CHANGES.txt

@@ -133,6 +133,9 @@ BUGFIXES:
   ZOOKEEPER-582. ZooKeeper can revert to old data when a snapshot is created
   ZOOKEEPER-582. ZooKeeper can revert to old data when a snapshot is created
   outside of normal processing (ben reed and mahadev via mahadev)
   outside of normal processing (ben reed and mahadev via mahadev)
 
 
+  ZOOKEEPER-597. ASyncHammerTest is failing intermittently on hudson trunk
+  (Patrick Hunt via mahadev)
+
 IMPROVEMENTS:
 IMPROVEMENTS:
   ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to
   ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to
   "socket reuse" and failure to close client (phunt via mahadev)
   "socket reuse" and failure to close client (phunt via mahadev)

+ 34 - 4
src/java/test/org/apache/zookeeper/test/ClientBase.java

@@ -28,6 +28,8 @@ import java.lang.management.OperatingSystemMXBean;
 import java.net.Socket;
 import java.net.Socket;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.LinkedList;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeoutException;
@@ -453,6 +455,20 @@ public abstract class ClientBase extends TestCase {
         return d.delete();
         return d.delete();
     }
     }
 
 
+    private static void logAllStackTraces() {
+        StringBuffer sb = new StringBuffer();
+        sb.append("Starting logAllStackTraces()\n");
+        Map<Thread, StackTraceElement[]> threads = Thread.getAllStackTraces();
+        for (Entry<Thread, StackTraceElement[]> e: threads.entrySet()) {
+            sb.append("Thread " + e.getKey().getName() + "\n");
+            for (StackTraceElement elem: e.getValue()) {
+                sb.append("\tat " + elem + "\n");
+            }
+        }
+        sb.append("Ending logAllStackTraces()\n");
+        LOG.error(sb.toString());
+    }
+
     /*
     /*
      * Verify that all of the servers see the same number of nodes
      * Verify that all of the servers see the same number of nodes
      * at the root
      * at the root
@@ -465,15 +481,24 @@ public abstract class ClientBase extends TestCase {
         // run through till the counts no longer change on each server
         // run through till the counts no longer change on each server
         // max 15 tries, with 2 second sleeps, so approx 30 seconds
         // max 15 tries, with 2 second sleeps, so approx 30 seconds
         int[] counts = new int[parts.length];
         int[] counts = new int[parts.length];
+        int failed = 0;
         for (int j = 0; j < 100; j++) {
         for (int j = 0; j < 100; j++) {
             int newcounts[] = new int[parts.length];
             int newcounts[] = new int[parts.length];
             int i = 0;
             int i = 0;
             for (String hp : parts) {
             for (String hp : parts) {
-                ZooKeeper zk = createClient(hp);
                 try {
                 try {
-                    newcounts[i++] = zk.getChildren("/", false).size();
-                } finally {
-                    zk.close();
+                    ZooKeeper zk = createClient(hp);
+                   
+                    try {
+                        newcounts[i++] = zk.getChildren("/", false).size();
+                    } finally {
+                        zk.close();
+                    }
+                } catch (Throwable t) {
+                    failed++;
+                    // if session creation fails dump the thread stack
+                    // and try the next server
+                    logAllStackTraces();
                 }
                 }
             }
             }
 
 
@@ -486,6 +511,11 @@ public abstract class ClientBase extends TestCase {
                 counts = newcounts;
                 counts = newcounts;
                 Thread.sleep(10000);
                 Thread.sleep(10000);
             }
             }
+            
+            // don't keep this up too long, will assert false below
+            if (failed > 10) {
+                break;
+            }
         }
         }
 
 
         // verify all the servers reporting same number of nodes
         // verify all the servers reporting same number of nodes