Wei-Chiu Chuang created HDFS-11254:
--------------------------------------
Summary: Failover may fail if loading edits takes too long
Key: HDFS-11254
URL: https://issues.apache.org/jira/browse/HDFS-11254
Project: Hadoop HDFS
Issue Type: Bug
Components: namenode
Affects Versions: 2.6.0
Reporter: Wei-Chiu Chuang
Priority: Critical
Fix For: 2.9.0, 3.0.0-beta1
We found Standby NameNode crashed when it tried to transition from standby to
active. This issue is similar to HDFS-11225 in nature.
The root cause is all IPC threads were blocked, so ZKFC connection to NN timed
out. In particular, when it crashed, we saw a few threads blocked on this
thread:
{noformat}
Thread 188 (IPC Server handler 25 on 8022):
State: RUNNABLE
Blocked count: 278
Waited count: 17419
Stack:
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuotaRecursively(FSImage.java:886)
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuotaRecursively(FSImage.java:887)
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuotaRecursively(FSImage.java:887)
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuotaRecursively(FSImage.java:887)
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuotaRecursively(FSImage.java:887)
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuotaRecursively(FSImage.java:887)
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuotaRecursively(FSImage.java:887)
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuotaRecursively(FSImage.java:887)
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuotaRecursively(FSImage.java:887)
org.apache.hadoop.hdfs.server.namenode.FSImage.updateCountForQuota(FSImage.java:875)
org.apache.hadoop.hdfs.server.namenode.FSImage.loadEdits(FSImage.java:860)
org.apache.hadoop.hdfs.server.namenode.FSImage.loadEdits(FSImage.java:827)
org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.doTailEdits(EditLogTailer.java:232)
org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$1.run(EditLogTailer.java:188)
org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$1.run(EditLogTailer.java:182)
java.security.AccessController.doPrivileged(Native Method)
javax.security.auth.Subject.doAs(Subject.java:415)
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1709)
org.apache.hadoop.security.SecurityUtil.doAsUser(SecurityUtil.java:477)
org.apache.hadoop.security.SecurityUtil.doAsLoginUser(SecurityUtil.java:458)
{noformat}
This thread is part of {{FsImage#loadEdits}} when the NameNode failed over.
We also found the following edit logs was rejected after journal node advanced
epoch, which implies a failed transitionToActive request.
{noformat}
10.10.17.1:8485: IPC's epoch 11 is less than the last promised epoch 12
at
org.apache.hadoop.hdfs.qjournal.server.Journal.checkRequest(Journal.java:429)
at
org.apache.hadoop.hdfs.qjournal.server.Journal.startLogSegment(Journal.java:513)
at
org.apache.hadoop.hdfs.qjournal.server.JournalNodeRpcServer.startLogSegment(JournalNodeRpcServer.java:162)
at
org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolServerSideTranslatorPB.startLogSegment(QJournalProtocolServerSideTranslatorPB.java:198)
at
org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos$QJournalProtocolService$2.callBlockingMethod(QJournalProtocolProtos.java:25425)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1073)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:415)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1709)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
at
org.apache.hadoop.hdfs.qjournal.client.QuorumException.create(QuorumException.java:81)
at
org.apache.hadoop.hdfs.qjournal.client.QuorumCall.rethrowException(QuorumCall.java:223)
at
org.apache.hadoop.hdfs.qjournal.client.AsyncLoggerSet.waitForWriteQuorum(AsyncLoggerSet.java:142)
at
org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager.startLogSegment(QuorumJournalManager.java:408)
at
org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.startLogSegment(JournalSet.java:107)
at
org.apache.hadoop.hdfs.server.namenode.JournalSet$3.apply(JournalSet.java:222)
at
org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
at
org.apache.hadoop.hdfs.server.namenode.JournalSet.startLogSegment(JournalSet.java:219)
at
org.apache.hadoop.hdfs.server.namenode.FSEditLog.startLogSegment(FSEditLog.java:1206)
at
org.apache.hadoop.hdfs.server.namenode.FSEditLog.openForWrite(FSEditLog.java:316)
at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startActiveServices(FSNamesystem.java:1265)
at
org.apache.hadoop.hdfs.server.namenode.NameNode$NameNodeHAContext.startActiveServices(NameNode.java:1767)
at
org.apache.hadoop.hdfs.server.namenode.ha.ActiveState.enterState(ActiveState.java:61)
at
org.apache.hadoop.hdfs.server.namenode.ha.HAState.setStateInternal(HAState.java:64)
at
org.apache.hadoop.hdfs.server.namenode.ha.StandbyState.setState(StandbyState.java:49)
at
org.apache.hadoop.hdfs.server.namenode.NameNode.transitionToActive(NameNode.java:1640)
at
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.transitionToActive(NameNodeRpcServer.java:1375)
at
org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB.transitionToActive(HAServiceProtocolServerSideTranslatorPB.java:107)
at
org.apache.hadoop.ha.proto.HAServiceProtocolProtos$HAServiceProtocolService$2.callBlockingMethod(HAServiceProtocolProtos.java:4460)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1073)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:415)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1709)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
{noformat}
We found the threads that waited on NameNodeRpcServer lock for the
transitionToActive thread called {{NameNodeRpcServer#getServiceStatus}}. Is it
possible to make this method unsynchronized? Furthermore, is it necessary to
synchronize on this object for other NameNodeRpcServer methods? (monitorHealth,
transitionToActive, transitionToStandby, getServiceStatus) Is it possible to
make {{FSImage.updateCountForQuotaRecursively}} faster?
Despite this issue resulted in failed failover, I am setting priority to
critical instead of blocker, because a possible workaround is extending ZKFC
socket time out.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]