소스 검색

HDFS-6184. Capture NN's thread dump when it fails over. Contributed by Ming Ma.

Akira Ajisaka 10 년 전
부모
커밋
2463666ecb

+ 2 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java

@@ -844,12 +844,11 @@ public abstract class ZKFailoverController {
    * @return the last health state passed to the FC
    * by the HealthMonitor.
    */
-  @VisibleForTesting
-  synchronized State getLastHealthState() {
+  protected synchronized State getLastHealthState() {
     return lastHealthState;
   }
 
-  private synchronized void setLastHealthState(HealthMonitor.State newState) {
+  protected synchronized void setLastHealthState(HealthMonitor.State newState) {
     LOG.info("Local service " + localTarget +
         " entered state: " + newState);
     lastHealthState = newState;

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

@@ -546,6 +546,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8255. Rename getBlockReplication to getPreferredBlockReplication.
     (Contributed by Zhe Zhang)
 
+    HDFS-6184. Capture NN's thread dump when it fails over.
+    (Ming Ma via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -543,7 +543,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_HA_AUTO_FAILOVER_ENABLED_DEFAULT = false;
   public static final String DFS_HA_ZKFC_PORT_KEY = "dfs.ha.zkfc.port";
   public static final int DFS_HA_ZKFC_PORT_DEFAULT = 8019;
-  
+  public static final String DFS_HA_ZKFC_NN_HTTP_TIMEOUT_KEY = "dfs.ha.zkfc.nn.http.timeout.ms";
+  public static final int DFS_HA_ZKFC_NN_HTTP_TIMEOUT_KEY_DEFAULT = 20000;
+
   // Security-related configs
   public static final String DFS_ENCRYPT_DATA_TRANSFER_KEY = "dfs.encrypt.data.transfer";
   public static final boolean DFS_ENCRYPT_DATA_TRANSFER_DEFAULT = false;

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java

@@ -20,15 +20,20 @@ package org.apache.hadoop.hdfs.tools;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
+import java.net.URL;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAServiceTarget;
+import org.apache.hadoop.ha.HealthMonitor;
 import org.apache.hadoop.ha.ZKFailoverController;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -37,6 +42,7 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.proto.HAZKInfoProtos.ActiveNodeInfo;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
@@ -57,6 +63,9 @@ public class DFSZKFailoverController extends ZKFailoverController {
   /* the same as superclass's localTarget, but with the more specfic NN type */
   private final NNHAServiceTarget localNNTarget;
 
+  // This is used only for unit tests
+  private boolean isThreadDumpCaptured = false;
+
   @Override
   protected HAServiceTarget dataToTarget(byte[] data) {
     ActiveNodeInfo proto;
@@ -201,4 +210,55 @@ public class DFSZKFailoverController extends ZKFailoverController {
     LOG.warn(msg);
     throw new AccessControlException(msg);
   }
+
+  /**
+   * capture local NN's thread dump and write it to ZKFC's log.
+   */
+  private void getLocalNNThreadDump() {
+    isThreadDumpCaptured = false;
+    // We use the same timeout value for both connection establishment
+    // timeout and read timeout.
+    int httpTimeOut = conf.getInt(
+        DFSConfigKeys.DFS_HA_ZKFC_NN_HTTP_TIMEOUT_KEY,
+        DFSConfigKeys.DFS_HA_ZKFC_NN_HTTP_TIMEOUT_KEY_DEFAULT);
+    if (httpTimeOut == 0) {
+      // If timeout value is set to zero, the feature is turned off.
+      return;
+    }
+    try {
+      String stacksUrl = DFSUtil.getInfoServer(localNNTarget.getAddress(),
+          conf, DFSUtil.getHttpClientScheme(conf)) + "/stacks";
+      URL url = new URL(stacksUrl);
+      HttpURLConnection conn = (HttpURLConnection)url.openConnection();
+      conn.setReadTimeout(httpTimeOut);
+      conn.setConnectTimeout(httpTimeOut);
+      conn.connect();
+      ByteArrayOutputStream out = new ByteArrayOutputStream();
+      IOUtils.copyBytes(conn.getInputStream(), out, 4096, true);
+      StringBuilder localNNThreadDumpContent =
+          new StringBuilder("-- Local NN thread dump -- \n");
+      localNNThreadDumpContent.append(out);
+      localNNThreadDumpContent.append("\n -- Local NN thread dump -- ");
+      LOG.info(localNNThreadDumpContent);
+      isThreadDumpCaptured = true;
+    } catch (IOException e) {
+      LOG.warn("Can't get local NN thread dump due to " + e.getMessage());
+    }
+  }
+
+  @Override
+  protected synchronized void setLastHealthState(HealthMonitor.State newState) {
+    super.setLastHealthState(newState);
+    // Capture local NN thread dump when the target NN health state changes.
+    if (getLastHealthState() == HealthMonitor.State.SERVICE_NOT_RESPONDING ||
+        getLastHealthState() == HealthMonitor.State.SERVICE_UNHEALTHY) {
+      getLocalNNThreadDump();
+    }
+  }
+
+  @VisibleForTesting
+  boolean isThreadDumpCaptured() {
+    return isThreadDumpCaptured;
+  }
+
 }

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -2334,4 +2334,15 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.ha.zkfc.nn.http.timeout.ms</name>
+  <value>20000</value>
+  <description>
+    The HTTP connection and read timeout value (unit is ms ) when DFS ZKFC
+    tries to get local NN thread dump after local NN becomes
+    SERVICE_NOT_RESPONDING or SERVICE_UNHEALTHY.
+    If it is set to zero, DFS ZKFC won't get local NN thread dump.
+  </description>
+</property>
+
 </configuration>

+ 23 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode.ha;
+package org.apache.hadoop.hdfs.tools;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -36,9 +36,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.tools.DFSHAAdmin;
-import org.apache.hadoop.hdfs.tools.DFSZKFailoverController;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeResourceChecker;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
@@ -47,6 +47,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import com.google.common.base.Supplier;
+import org.mockito.Mockito;
 
 public class TestDFSZKFailoverController extends ClientBaseWithFixes {
   private Configuration conf;
@@ -122,7 +123,23 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
       ctx.stop();
     }
   }
-  
+
+  /**
+   * Test that thread dump is captured after NN state changes.
+   */
+  @Test(timeout=60000)
+  public void testThreadDumpCaptureAfterNNStateChange() throws Exception {
+    NameNodeResourceChecker mockResourceChecker = Mockito.mock(
+        NameNodeResourceChecker.class);
+    Mockito.doReturn(false).when(mockResourceChecker).hasAvailableDiskSpace();
+    cluster.getNameNode(0).getNamesystem()
+        .setNNResourceChecker(mockResourceChecker);
+    waitForHAState(0, HAServiceState.STANDBY);
+    while (!thr1.zkfc.isThreadDumpCaptured()) {
+      Thread.sleep(1000);
+    }
+  }
+
   /**
    * Test that automatic failover is triggered by shutting the
    * active NN down.
@@ -131,7 +148,7 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
   public void testFailoverAndBackOnNNShutdown() throws Exception {
     Path p1 = new Path("/dir1");
     Path p2 = new Path("/dir2");
-    
+
     // Write some data on the first NN
     fs.mkdirs(p1);
     // Shut it down, causing automatic failover
@@ -183,7 +200,7 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
     waitForHAState(0, HAServiceState.ACTIVE);
     waitForHAState(1, HAServiceState.STANDBY);
   }
-  
+
   private void waitForHAState(int nnidx, final HAServiceState state)
       throws TimeoutException, InterruptedException {
     final NameNode nn = cluster.getNameNode(nnidx);