[
https://issues.apache.org/jira/browse/HDFS-14961?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16977826#comment-16977826
]
Ayush Saxena commented on HDFS-14961:
-------------------------------------
Have uploaded a patch, with the fix as said above.
The test can be reproduced by putting a {{Thread.sleep(10000)}} just before {{
waitForHAState(1, HAServiceState.OBSERVER);}} (In most cases it works for me.)
Fix : Once the namenode is transitioned to OBSERVER, It ignores Request of
turning to STANDBY by ZKFC as OBSERVER isn't suppose to participate in election.
Have introduced a separate test to verify the exception.
The LOG for {{testManualFailoverWithDFSHAAdmin}} with the fix, in the failing
case looks like :
{noformat}
2019-11-20 01:39:21,219 [IPC Server handler 4 on default port 10022] INFO
namenode.FSNamesystem (FSNamesystem.java:startStandbyServices(1402)) - Starting
services required for observer state
2019-11-20 01:39:21,222 [IPC Server handler 4 on default port 10022] INFO
ha.EditLogTailer (EditLogTailer.java:<init>(205)) - Will roll logs on active
node every 120 seconds.
2019-11-20 01:39:21,670 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:21,824 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO zookeeper.ZooKeeper
(ZooKeeper.java:close(1422)) - Session: 0x10001326a8a0004 closed
2019-11-20 01:39:21,825 [IPC Server handler 2 on default port
10024-EventThread] WARN ha.ActiveStandbyElector
(ActiveStandbyElector.java:isStaleClient(1124)) - Ignoring stale result from
old client with sessionId 0x10001326a8a0004
2019-11-20 01:39:21,825 [IPC Server handler 2 on default port
10024-EventThread] INFO zookeeper.ClientCnxn (ClientCnxn.java:run(524)) -
EventThread shut down for session: 0x10001326a8a0004
2019-11-20 01:39:22,836 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:23,850 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:24,856 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:25,864 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:26,871 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:27,879 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:28,881 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:29,888 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:30,097 [ZKFC Delay timer #0] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:joinElection(300)) - Already in election. Not
re-connecting.
2019-11-20 01:39:30,752 [ZKFC Delay timer #0] INFO zookeeper.ZooKeeper
(ZooKeeper.java:<init>(868)) - Initiating client connection,
connectString=127.0.0.1:6958 sessionTimeout=10000
watcher=org.apache.hadoop.ha.ActiveStandbyElector$WatcherWithClientRef@2ebd10bf
2019-11-20 01:39:30,752 [ZKFC Delay timer #0] INFO zookeeper.ClientCnxnSocket
(ClientCnxnSocket.java:initProperties(237)) - jute.maxbuffer value is 4194304
Bytes
2019-11-20 01:39:30,753 [ZKFC Delay timer #0] INFO zookeeper.ClientCnxn
(ClientCnxn.java:initRequestTimeout(1653)) - zookeeper.request.timeout value is
0. feature enabled=
2019-11-20 01:39:30,754 [ZKFC Delay timer #0-SendThread(127.0.0.1:6958)] INFO
zookeeper.ClientCnxn (ClientCnxn.java:logStartConnect(1112)) - Opening socket
connection to server localhost/127.0.0.1:6958. Will not attempt to authenticate
using SASL (unknown error)
2019-11-20 01:39:30,754 [ZKFC Delay timer #0-SendThread(127.0.0.1:6958)] INFO
zookeeper.ClientCnxn (ClientCnxn.java:primeConnection(959)) - Socket connection
established, initiating session, client: /127.0.0.1:50524, server:
localhost/127.0.0.1:6958
2019-11-20 01:39:30,803 [ZKFC Delay timer #0-SendThread(127.0.0.1:6958)] INFO
zookeeper.ClientCnxn (ClientCnxn.java:onConnected(1394)) - Session
establishment complete on server localhost/127.0.0.1:6958, sessionid =
0x10001326a8a0005, negotiated timeout = 10000
2019-11-20 01:39:30,804 [ZKFC Delay timer #0-EventThread] INFO
ha.ActiveStandbyElector (ActiveStandbyElector.java:processWatchEvent(611)) -
Session connected.
2019-11-20 01:39:30,821 [ZKFC Delay timer #0-EventThread] INFO
ha.ZKFailoverController (ZKFailoverController.java:becomeStandby(491)) - ZK
Election indicated that NameNode at localhost/127.0.0.1:10022 should become
standby
2019-11-20 01:39:30,823 [IPC Server handler 1 on default port 10022] INFO
ipc.Server (Server.java:logException(2981)) - IPC Server handler 1 on default
port 10022, call Call#65 Retry#0
org.apache.hadoop.ha.HAServiceProtocol.transitionToStandby from
127.0.0.1:57646: org.apache.hadoop.security.AccessControlException: Request
from ZK failover controller at 127.0.0.1 denied since the namenode is in
Observer state.
2019-11-20 01:39:30,877 [ZKFC Delay timer #0-EventThread] ERROR
ha.ZKFailoverController (ZKFailoverController.java:becomeStandby(499)) -
Couldn't transition NameNode at localhost/127.0.0.1:10022 to standby state
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.AccessControlException):
Request from ZK failover controller at 127.0.0.1 denied since the namenode is
in Observer state.
at
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.transitionToStandby(NameNodeRpcServer.java:1797)
at
org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB.transitionToStandby(HAServiceProtocolServerSideTranslatorPB.java:124)
at
org.apache.hadoop.ha.proto.HAServiceProtocolProtos$HAServiceProtocolService$2.callBlockingMethod(HAServiceProtocolProtos.java:6220)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:529)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1070)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1007)
at org.apache.hadoop.ipc.Server$Call.run(Server.java:1)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1863)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2927)
at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1553)
at org.apache.hadoop.ipc.Client.call(Client.java:1499)
at org.apache.hadoop.ipc.Client.call(Client.java:1396)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:1)
at com.sun.proxy.$Proxy25.transitionToStandby(Unknown Source)
at
org.apache.hadoop.ha.protocolPB.HAServiceProtocolClientSideTranslatorPB.transitionToStandby(HAServiceProtocolClientSideTranslatorPB.java:113)
at
org.apache.hadoop.ha.ZKFailoverController.becomeStandby(ZKFailoverController.java:495)
at
org.apache.hadoop.ha.ZKFailoverController.access$1(ZKFailoverController.java:490)
at
org.apache.hadoop.ha.ZKFailoverController$ElectorCallbacks.becomeStandby(ZKFailoverController.java:932)
at
org.apache.hadoop.ha.ActiveStandbyElector.becomeStandby(ActiveStandbyElector.java:1001)
at
org.apache.hadoop.ha.ActiveStandbyElector.processResult(ActiveStandbyElector.java:488)
at
org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:642)
at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:510)
2019-11-20 01:39:30,891 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO ha.ActiveStandbyElector
(ActiveStandbyElector.java:quitElection(412)) - Yielding from election
2019-11-20 01:39:31,028 [Health Monitor for NameNode at
localhost/127.0.0.1:10022] INFO zookeeper.ZooKeeper
(ZooKeeper.java:close(1422)) - Session: 0x10001326a8a0005 closed
2019-11-20 01:39:31,028 [ZKFC Delay timer #0-EventThread] WARN
ha.ActiveStandbyElector (ActiveStandbyElector.java:isStaleClient(1124)) -
Ignoring stale result from old client with sessionId 0x10001326a8a0005
2019-11-20 01:39:31,029 [ZKFC Delay timer #0-EventThread] INFO
zookeeper.ClientCnxn (ClientCnxn.java:run(524)) - EventThread shut down for
session: 0x10001326a8a0005
{noformat}
[~elgoiri] can you help review.
> TestDFSZKFailoverController fails consistently
> ----------------------------------------------
>
> Key: HDFS-14961
> URL: https://issues.apache.org/jira/browse/HDFS-14961
> Project: Hadoop HDFS
> Issue Type: Bug
> Reporter: Íñigo Goiri
> Priority: Major
> Attachments: HDFS-14961-01.patch
>
>
> TestDFSZKFailoverController has been consistently failing with a time out
> waiting in testManualFailoverWithDFSHAAdmin(). In particular
> {{waitForHAState(1, HAServiceState.OBSERVER);}}.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]