[
https://issues.apache.org/jira/browse/HDFS-15235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17069461#comment-17069461
]
YCozy commented on HDFS-15235:
------------------------------
Thank you [~ayushtkn]! Upon further analysis we found that NN1 did become
active again. However, NN2 shouldn't be killed so that another failover can be
easily launched. Since we don't want NN1 to stay active at the first place,
leaving NN1 as the only running NN in the cluster doesn't really help.
I've made the title more accurate.
> Transient network failure during NameNode failover kills the NameNode
> ---------------------------------------------------------------------
>
> Key: HDFS-15235
> URL: https://issues.apache.org/jira/browse/HDFS-15235
> Project: Hadoop HDFS
> Issue Type: Bug
> Affects Versions: 3.3.0
> Reporter: YCozy
> Assignee: YCozy
> Priority: Major
> Attachments: HDFS-15235.001.patch
>
>
> We have an HA cluster with two NameNodes: an active NN1 and a standby NN2. At
> some point, NN1 becomes unhealthy and the admin tries to manually failover to
> NN2 by running command
> {code:java}
> $ hdfs haadmin -failover NN1 NN2
> {code}
> NN2 receives the request and becomes active:
> {code:java}
> 2020-03-24 00:24:56,412 INFO
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Stopping services
> started for standby state
> 2020-03-24 00:24:56,413 WARN
> org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer: Edit log tailer
> interrupted: sleep interrupted
> 2020-03-24 00:24:56,415 INFO
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Starting services
> required for active state
> 2020-03-24 00:24:56,417 INFO
> org.apache.hadoop.hdfs.server.namenode.FileJournalManager: Recovering
> unfinalized segments in /app/ha-name-dir-shared/current
> 2020-03-24 00:24:56,419 INFO
> org.apache.hadoop.hdfs.server.namenode.FileJournalManager: Recovering
> unfinalized segments in /app/nn2/name/current
> 2020-03-24 00:24:56,419 INFO
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Catching up to latest
> edits from old active before taking over writer role in edits logs
> 2020-03-24 00:24:56,435 INFO org.apache.hadoop.hdfs.server.namenode.FSImage:
> Reading
> org.apache.hadoop.hdfs.server.namenode.RedundantEditLogInputStream@7c3095fa
> expecting start txid #1
> 2020-03-24 00:24:56,436 INFO org.apache.hadoop.hdfs.server.namenode.FSImage:
> Start loading edits file
> /app/ha-name-dir-shared/current/edits_0000000000000000001-0000000000000000019
> maxTxnsToRead = 9223372036854775807
> 2020-03-24 00:24:56,441 INFO
> org.apache.hadoop.hdfs.server.namenode.RedundantEditLogInputStream:
> Fast-forwarding stream
> '/app/ha-name-dir-shared/current/edits_0000000000000000001-0000000000000000019'
> to transaction ID 1
> 2020-03-24 00:24:56,567 INFO org.apache.hadoop.hdfs.server.namenode.FSImage:
> Loaded 1 edits file(s) (the last named
> /app/ha-name-dir-shared/current/edits_0000000000000000001-0000000000000000019)
> of total size 1305.0, total edits 19.0, total load time 109.0 ms
> 2020-03-24 00:24:56,567 INFO
> org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager: Marking all
> datanodes as stale
> 2020-03-24 00:24:56,568 INFO
> org.apache.hadoop.hdfs.server.blockmanagement.BlockManager: Processing 4
> messages from DataNodes that were previously queued during standby state
> 2020-03-24 00:24:56,569 INFO
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Reprocessing replication
> and invalidation queues
> 2020-03-24 00:24:56,569 INFO
> org.apache.hadoop.hdfs.server.blockmanagement.BlockManager: initializing
> replication queues
> 2020-03-24 00:24:56,570 INFO
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Will take over writing
> edit logs at txnid 20
> 2020-03-24 00:24:56,571 INFO
> org.apache.hadoop.hdfs.server.namenode.FSEditLog: Starting log segment at 20
> 2020-03-24 00:24:56,812 INFO
> org.apache.hadoop.hdfs.server.namenode.FSDirectory: Initializing quota with 4
> thread(s)
> 2020-03-24 00:24:56,819 INFO
> org.apache.hadoop.hdfs.server.namenode.FSDirectory: Quota initialization
> completed in 6 millisecondsname space=3storage space=24690storage
> types=RAM_DISK=0, SSD=0, DISK=0, ARCHIVE=0, PROVIDED=0
> 2020-03-24 00:24:56,827 INFO
> org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor:
> Starting CacheReplicationMonitor with interval 30000 milliseconds
> {code}
> But NN2 fails to send back the RPC response because of temporary network
> partitioning.
> {code:java}
> java.io.EOFException: End of File Exception between local host is:
> "24e7b5a52e85/172.17.0.2"; destination host is: "127.0.0.3":8180; :
> java.io.EOFException; For more details see:
> http://wiki.apache.org/hadoop/EOFException
> at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> Method)
> at
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
> at
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
> at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
> at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:837)
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:791)
> at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1597)
> at org.apache.hadoop.ipc.Client.call(Client.java:1539)
> at org.apache.hadoop.ipc.Client.call(Client.java:1436)
> at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
> at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:118)
> at com.sun.proxy.$Proxy8.transitionToActive(Unknown Source)
> at
> org.apache.hadoop.ha.protocolPB.HAServiceProtocolClientSideTranslatorPB.transitionToActive(HAServiceProtocolClientSideTranslatorPB.java:101)
> at
> org.apache.hadoop.ha.HAServiceProtocolHelper.transitionToActive(HAServiceProtocolHelper.java:48)
> at
> org.apache.hadoop.ha.FailoverController.failover(FailoverController.java:226)
> at
> org.apache.hadoop.hdfs.tools.DFSHAAdmin.failover(DFSHAAdmin.java:319)
> at org.apache.hadoop.hdfs.tools.DFSHAAdmin.runCmd(DFSHAAdmin.java:217)
> at org.apache.hadoop.ha.HAAdmin.run(HAAdmin.java:346)
> at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
> at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
> at org.apache.hadoop.hdfs.tools.DFSHAAdmin.main(DFSHAAdmin.java:329)
> Caused by: java.io.EOFException
> at java.io.DataInputStream.readInt(DataInputStream.java:392)
> at
> org.apache.hadoop.ipc.Client$IpcStreams.readResponse(Client.java:1912)
> at
> org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse(Client.java:1222)
> at org.apache.hadoop.ipc.Client$Connection.run(Client.java:1092){code}
> As a result, the haadmin command thinks the failover fails and starts to
> fence NN2 using SshFenceByTcpPort. Unfortunately, the network partition
> stops, so the fencing succeeds. As a result, NN2 is killed and the cluster
> has no healthy NameNode.
> {code:java}
> 2020-03-24 00:24:57,622 INFO ha.SshFenceByTcpPort: Successfully killed
> process that was listening on port 8180
> 2020-03-24 00:24:57,623 INFO SshFenceByTcpPort.jsch: Disconnecting from
> 127.0.0.3 port 22
> 2020-03-24 00:24:57,627 INFO ha.NodeFencer: ====== Fencing successful by
> method org.apache.hadoop.ha.SshFenceByTcpPort(null) ======{code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]