[ 
https://issues.apache.org/jira/browse/HDFS-9126?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

zengyongping updated HDFS-9126:
-------------------------------
    Description: 
In our product Hadoop cluster,when active namenode begin download/transfer 
fsimage from standby namenode.some times zkfc monitor health of NameNode socket 
timeout,zkfs judge active namenode status SERVICE_NOT_RESPONDING ,happen hadoop 
namenode ha failover,fence old active namenode.

zkfc logs:
2015-09-24 11:44:44,739 WARN org.apache.hadoop.ha.HealthMonitor: 
Transport-level exception trying to monitor health of NameNode at 
hostname1/192.168.10.11:8020: Call From hostname1/192.168.10.11 to 
hostname1:8020 failed on socket timeout exception: 
java.net.SocketTimeoutException: 45000 millis timeout while waiting for channel 
to be ready for read. ch : java.nio.channels.SocketChannel[connected 
local=/192.168.10.11:22614 remote=hostname1/192.168.10.11:8020]; For more 
details see:  http://wiki.apache.org/hadoop/SocketTimeout
2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.HealthMonitor: Entering state 
SERVICE_NOT_RESPONDING
2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.ZKFailoverController: Local 
service NameNode at hostname1/192.168.10.11:8020 entered state: 
SERVICE_NOT_RESPONDING
2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.ZKFailoverController: 
Quitting master election for NameNode at hostname1/192.168.10.11:8020 and 
marking that fencing is necessary
2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.ActiveStandbyElector: 
Yielding from election
2015-09-24 11:44:44,761 INFO org.apache.zookeeper.ZooKeeper: Session: 
0x54d81348fe503e3 closed
2015-09-24 11:44:44,761 WARN org.apache.hadoop.ha.ActiveStandbyElector: 
Ignoring stale result from old client with sessionId 0x54d81348fe503e3
2015-09-24 11:44:44,764 INFO org.apache.zookeeper.ClientCnxn: EventThread shut 
down

namenode logs:
2015-09-24 11:43:34,074 INFO 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Roll Edit Log from 
192.168.10.12
2015-09-24 11:43:34,074 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Rolling edit logs
2015-09-24 11:43:34,075 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Ending log segment 2317430129
2015-09-24 11:43:34,253 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Number of transactions: 272988 Total time for transactions(ms): 5502 Number of 
transactions batched in Syncs: 146274 Number of syncs: 32375 SyncTimes(ms): 
274465 319599
2015-09-24 11:43:46,005 INFO 
org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor: 
Rescanning after 30000 milliseconds
2015-09-24 11:44:21,054 WARN 
org.apache.hadoop.hdfs.server.blockmanagement.BlockManager: 
PendingReplicationMonitor timed out blk_1185804191_112164210
2015-09-24 11:44:36,076 INFO 
org.apache.hadoop.hdfs.server.namenode.FileJournalManager: Finalizing edits 
file 
/software/data/hadoop-data/hdfs/namenode/current/edits_inprogress_0000000002317430129
 -> 
/software/data/hadoop-data/hdfs/namenode/current/edits_0000000002317430129-0000000002317703116
2015-09-24 11:44:36,077 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Starting log segment at 2317703117
2015-09-24 11:45:38,008 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Number of transactions: 1 Total time for transactions(ms): 0 Number of 
transactions batched in Syncs: 0 Number of syncs: 0 SyncTimes(ms): 0 61585
2015-09-24 11:45:38,009 INFO 
org.apache.hadoop.hdfs.server.namenode.TransferFsImage: Transfer took 222.88s 
at 63510.29 KB/s
2015-09-24 11:45:38,009 INFO 
org.apache.hadoop.hdfs.server.namenode.TransferFsImage: Downloaded file 
fsimage.ckpt_0000000002317430128 size 14495092105 bytes.
2015-09-24 11:45:38,416 WARN 
org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Remote journal 
192.168.10.13:8485 failed to write txns 2317703117-2317703117. Will try to 
write to this JN again after the next log roll.
org.apache.hadoop.ipc.RemoteException(java.io.IOException): IPC's epoch 44 is 
less than the last promised epoch 45
        at 
org.apache.hadoop.hdfs.qjournal.server.Journal.checkRequest(Journal.java:414)
        at 
org.apache.hadoop.hdfs.qjournal.server.Journal.checkWriteRequest(Journal.java:442)
        at 
org.apache.hadoop.hdfs.qjournal.server.Journal.journal(Journal.java:342)
        at 
org.apache.hadoop.hdfs.qjournal.server.JournalNodeRpcServer.journal(JournalNodeRpcServer.java:148)
        at 
org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolServerSideTranslatorPB.journal(QJournalProtocolServerSideTranslatorPB.java:158)
        at 
org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos$QJournalProtocolService$2.callBlockingMethod(QJournalProtocolProtos.java:25421)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:619)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:962)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2039)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2035)
        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:1628)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2033)
        at org.apache.hadoop.ipc.Client.call(Client.java:1468)
        at org.apache.hadoop.ipc.Client.call(Client.java:1399)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
        at com.sun.proxy.$Proxy9.journal(Unknown Source)
        at 
org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolTranslatorPB.journal(QJournalProtocolTranslatorPB.java:167)
        at 
org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel$7.call(IPCLoggerChannel.java:385)
        at 
org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel$7.call(IPCLoggerChannel.java:378)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
                
#Similar log like above 

2015-09-24 11:45:38,418 WARN 
org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
QuorumOutputStream starting at txid 2317703117
2015-09-24 11:45:38,505 INFO org.apache.hadoop.util.ExitUtil: Exiting with 
status 1
2015-09-24 11:45:38,549 INFO org.apache.hadoop.hdfs.server.namenode.NameNode: 
SHUTDOWN_MSG:
/************************************************************
SHUTDOWN_MSG: Shutting down NameNode at hostname2/192.168.10.11


  was:
In our product Hadoop cluster,when active namenode begin download/transfer 
fsimage from standby namenode.some times zkfc monitor health of NameNode socket 
timeout,zkfs judge active namenode status SERVICE_NOT_RESPONDING ,happen hadoop 
namenode ha failover,fence old active namenode.

zkfc logs:
2015-09-24 11:44:44,739 WARN org.apache.hadoop.ha.HealthMonitor: 
Transport-level exception trying to monitor health of NameNode at 
hostname1/192.168.10.11:8020: Call From hostname1/192.168.10.11 to 
hostname1:8020 failed on socket timeout exception: 
java.net.SocketTimeoutException: 45000 millis timeout while waiting for channel 
to be ready for read. ch : java.nio.channels.SocketChannel[connected 
local=/192.168.10.11:22614 remote=hostname1/192.168.10.11:8020]; For more 
details see:  http://wiki.apache.org/hadoop/SocketTimeout
2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.HealthMonitor: Entering state 
SERVICE_NOT_RESPONDING
2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.ZKFailoverController: Local 
service NameNode at hostname1/192.168.10.11:8020 entered state: 
SERVICE_NOT_RESPONDING
2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.ZKFailoverController: 
Quitting master election for NameNode at hostname1/192.168.10.11:8020 and 
marking that fencing is necessary
2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.ActiveStandbyElector: 
Yielding from election
2015-09-24 11:44:44,761 INFO org.apache.zookeeper.ZooKeeper: Session: 
0x54d81348fe503e3 closed
2015-09-24 11:44:44,761 WARN org.apache.hadoop.ha.ActiveStandbyElector: 
Ignoring stale result from old client with sessionId 0x54d81348fe503e3
2015-09-24 11:44:44,764 INFO org.apache.zookeeper.ClientCnxn: EventThread shut 
down



> namenode crash in fsimage download/transfer
> -------------------------------------------
>
>                 Key: HDFS-9126
>                 URL: https://issues.apache.org/jira/browse/HDFS-9126
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: namenode
>    Affects Versions: 2.6.0
>         Environment: OS:Centos 6.5(final)
> Hadoop:2.6.0
> namenode ha base 5 journalnodes
>            Reporter: zengyongping
>            Priority: Critical
>
> In our product Hadoop cluster,when active namenode begin download/transfer 
> fsimage from standby namenode.some times zkfc monitor health of NameNode 
> socket timeout,zkfs judge active namenode status SERVICE_NOT_RESPONDING 
> ,happen hadoop namenode ha failover,fence old active namenode.
> zkfc logs:
> 2015-09-24 11:44:44,739 WARN org.apache.hadoop.ha.HealthMonitor: 
> Transport-level exception trying to monitor health of NameNode at 
> hostname1/192.168.10.11:8020: Call From hostname1/192.168.10.11 to 
> hostname1:8020 failed on socket timeout exception: 
> java.net.SocketTimeoutException: 45000 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/192.168.10.11:22614 remote=hostname1/192.168.10.11:8020]; For more 
> details see:  http://wiki.apache.org/hadoop/SocketTimeout
> 2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.HealthMonitor: Entering 
> state SERVICE_NOT_RESPONDING
> 2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.ZKFailoverController: Local 
> service NameNode at hostname1/192.168.10.11:8020 entered state: 
> SERVICE_NOT_RESPONDING
> 2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.ZKFailoverController: 
> Quitting master election for NameNode at hostname1/192.168.10.11:8020 and 
> marking that fencing is necessary
> 2015-09-24 11:44:44,740 INFO org.apache.hadoop.ha.ActiveStandbyElector: 
> Yielding from election
> 2015-09-24 11:44:44,761 INFO org.apache.zookeeper.ZooKeeper: Session: 
> 0x54d81348fe503e3 closed
> 2015-09-24 11:44:44,761 WARN org.apache.hadoop.ha.ActiveStandbyElector: 
> Ignoring stale result from old client with sessionId 0x54d81348fe503e3
> 2015-09-24 11:44:44,764 INFO org.apache.zookeeper.ClientCnxn: EventThread 
> shut down
> namenode logs:
> 2015-09-24 11:43:34,074 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Roll Edit Log from 
> 192.168.10.12
> 2015-09-24 11:43:34,074 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog: Rolling edit logs
> 2015-09-24 11:43:34,075 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog: Ending log segment 
> 2317430129
> 2015-09-24 11:43:34,253 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog: Number of transactions: 
> 272988 Total time for transactions(ms): 5502 Number of transactions batched 
> in Syncs: 146274 Number of syncs: 32375 SyncTimes(ms): 274465 319599
> 2015-09-24 11:43:46,005 INFO 
> org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor: 
> Rescanning after 30000 milliseconds
> 2015-09-24 11:44:21,054 WARN 
> org.apache.hadoop.hdfs.server.blockmanagement.BlockManager: 
> PendingReplicationMonitor timed out blk_1185804191_112164210
> 2015-09-24 11:44:36,076 INFO 
> org.apache.hadoop.hdfs.server.namenode.FileJournalManager: Finalizing edits 
> file 
> /software/data/hadoop-data/hdfs/namenode/current/edits_inprogress_0000000002317430129
>  -> 
> /software/data/hadoop-data/hdfs/namenode/current/edits_0000000002317430129-0000000002317703116
> 2015-09-24 11:44:36,077 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog: Starting log segment at 
> 2317703117
> 2015-09-24 11:45:38,008 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog: Number of transactions: 1 
> Total time for transactions(ms): 0 Number of transactions batched in Syncs: 0 
> Number of syncs: 0 SyncTimes(ms): 0 61585
> 2015-09-24 11:45:38,009 INFO 
> org.apache.hadoop.hdfs.server.namenode.TransferFsImage: Transfer took 222.88s 
> at 63510.29 KB/s
> 2015-09-24 11:45:38,009 INFO 
> org.apache.hadoop.hdfs.server.namenode.TransferFsImage: Downloaded file 
> fsimage.ckpt_0000000002317430128 size 14495092105 bytes.
> 2015-09-24 11:45:38,416 WARN 
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Remote journal 
> 192.168.10.13:8485 failed to write txns 2317703117-2317703117. Will try to 
> write to this JN again after the next log roll.
> org.apache.hadoop.ipc.RemoteException(java.io.IOException): IPC's epoch 44 is 
> less than the last promised epoch 45
>         at 
> org.apache.hadoop.hdfs.qjournal.server.Journal.checkRequest(Journal.java:414)
>         at 
> org.apache.hadoop.hdfs.qjournal.server.Journal.checkWriteRequest(Journal.java:442)
>         at 
> org.apache.hadoop.hdfs.qjournal.server.Journal.journal(Journal.java:342)
>         at 
> org.apache.hadoop.hdfs.qjournal.server.JournalNodeRpcServer.journal(JournalNodeRpcServer.java:148)
>         at 
> org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolServerSideTranslatorPB.journal(QJournalProtocolServerSideTranslatorPB.java:158)
>         at 
> org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos$QJournalProtocolService$2.callBlockingMethod(QJournalProtocolProtos.java:25421)
>         at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:619)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:962)
>         at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2039)
>         at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2035)
>         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:1628)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2033)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1468)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1399)
>         at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
>         at com.sun.proxy.$Proxy9.journal(Unknown Source)
>         at 
> org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolTranslatorPB.journal(QJournalProtocolTranslatorPB.java:167)
>         at 
> org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel$7.call(IPCLoggerChannel.java:385)
>         at 
> org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel$7.call(IPCLoggerChannel.java:378)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:745)
>               
> #Similar log like above 
> 2015-09-24 11:45:38,418 WARN 
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
> QuorumOutputStream starting at txid 2317703117
> 2015-09-24 11:45:38,505 INFO org.apache.hadoop.util.ExitUtil: Exiting with 
> status 1
> 2015-09-24 11:45:38,549 INFO org.apache.hadoop.hdfs.server.namenode.NameNode: 
> SHUTDOWN_MSG:
> /************************************************************
> SHUTDOWN_MSG: Shutting down NameNode at hostname2/192.168.10.11



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to