Browse Source

HDFS-4006. TestCheckpoint#testSecondaryHasVeryOutOfDateImage occasionally fails due to unexpected exit. Contributed by Todd Lipcon

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1395388 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 12 years ago
parent
commit
99e7354cd5

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -75,6 +75,9 @@ Release 2.0.3-alpha - Unreleased
 
 
     HDFS-3999. HttpFS OPEN operation expects len parameter, it should be length. (tucu)
     HDFS-3999. HttpFS OPEN operation expects len parameter, it should be length. (tucu)
 
 
+    HDFS-4006. TestCheckpoint#testSecondaryHasVeryOutOfDateImage
+    occasionally fails due to unexpected exit. (todd via eli)
+
 Release 2.0.2-alpha - 2012-09-07 
 Release 2.0.2-alpha - 2012-09-07 
   
   
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 23 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -76,6 +76,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList;
 
 
 /**********************************************************
 /**********************************************************
@@ -120,6 +121,8 @@ public class SecondaryNameNode implements Runnable {
   private CheckpointConf checkpointConf;
   private CheckpointConf checkpointConf;
   private FSNamesystem namesystem;
   private FSNamesystem namesystem;
 
 
+  private Thread checkpointThread;
+
 
 
   @Override
   @Override
   public String toString() {
   public String toString() {
@@ -275,6 +278,15 @@ public class SecondaryNameNode implements Runnable {
    */
    */
   public void shutdown() {
   public void shutdown() {
     shouldRun = false;
     shouldRun = false;
+    if (checkpointThread != null) {
+      checkpointThread.interrupt();
+      try {
+        checkpointThread.join(10000);
+      } catch (InterruptedException e) {
+        LOG.info("Interrupted waiting to join on checkpointer thread");
+        Thread.currentThread().interrupt(); // maintain status
+      }
+    }
     try {
     try {
       if (infoServer != null) infoServer.stop();
       if (infoServer != null) infoServer.stop();
     } catch (Exception e) {
     } catch (Exception e) {
@@ -588,12 +600,20 @@ public class SecondaryNameNode implements Runnable {
       terminate(ret);
       terminate(ret);
     }
     }
 
 
-    // Create a never ending deamon
-    Daemon checkpointThread = new Daemon(secondary);
-    checkpointThread.start();
+    secondary.startCheckpointThread();
   }
   }
   
   
   
   
+  public void startCheckpointThread() {
+    Preconditions.checkState(checkpointThread == null,
+        "Should not already have a thread");
+    Preconditions.checkState(shouldRun, "shouldRun should be true");
+    
+    checkpointThread = new Daemon(this);
+    checkpointThread.start();
+  }
+
+
   /**
   /**
    * Container for parsed command-line options.
    * Container for parsed command-line options.
    */
    */

+ 21 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -30,6 +30,8 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -74,6 +76,7 @@ import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.ArgumentMatcher;
 import org.mockito.ArgumentMatcher;
@@ -127,7 +130,22 @@ public class TestCheckpoint {
     stm.close();
     stm.close();
   }
   }
   
   
-  
+  @After
+  public void checkForSNNThreads() {
+    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+    
+    ThreadInfo[] infos = threadBean.getThreadInfo(threadBean.getAllThreadIds(), 20);
+    for (ThreadInfo info : infos) {
+      if (info == null) continue;
+      LOG.info("Check thread: " + info.getThreadName());
+      if (info.getThreadName().contains("SecondaryNameNode")) {
+        fail("Leaked thread: " + info + "\n" +
+            Joiner.on("\n").join(info.getStackTrace()));
+      }
+    }
+    LOG.info("--------");
+  }
+
   static void checkFile(FileSystem fileSys, Path name, int repl)
   static void checkFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
     throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
@@ -1744,7 +1762,7 @@ public class TestCheckpoint {
   /**
   /**
    * Test that the 2NN triggers a checkpoint after the configurable interval
    * Test that the 2NN triggers a checkpoint after the configurable interval
    */
    */
-  @Test
+  @Test(timeout=30000)
   public void testCheckpointTriggerOnTxnCount() throws Exception {
   public void testCheckpointTriggerOnTxnCount() throws Exception {
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
     SecondaryNameNode secondary = null;
     SecondaryNameNode secondary = null;
@@ -1758,8 +1776,7 @@ public class TestCheckpoint {
           .format(true).build();
           .format(true).build();
       FileSystem fs = cluster.getFileSystem();
       FileSystem fs = cluster.getFileSystem();
       secondary = startSecondaryNameNode(conf);
       secondary = startSecondaryNameNode(conf);
-      Thread t = new Thread(secondary);
-      t.start();
+      secondary.startCheckpointThread();
       final NNStorage storage = secondary.getFSImage().getStorage();
       final NNStorage storage = secondary.getFSImage().getStorage();
 
 
       // 2NN should checkpoint at startup
       // 2NN should checkpoint at startup