HDFS-6184. Capture NN's thread dump when it fails over. Contributed by Ming Ma.
(cherry picked from commit 2463666ecb
)
This commit is contained in:
parent
d5755ba241
commit
7596c6e731
|
@ -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;
|
||||
|
|
|
@ -213,6 +213,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
|
||||
|
|
|
@ -545,7 +545,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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -2333,4 +2333,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>
|
||||
|
|
|
@ -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);
|
Loading…
Reference in New Issue