[jira] [Commented] (HDFS-15235) Transient network failure during NameNode failover kills the NameNode

2020-04-01 Thread YCozy (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-15235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17072805#comment-17072805
 ] 

YCozy commented on HDFS-15235:
--

Hello [~weichiu], thanks for looking into this!

For triggering this bug, the most critical step is to prevent NN2 from sending 
the response back. We have a tool that can reliably trigger this bug by 
simulating network partitions in a real cluster (i.e., not via unit tests).

On the other hand, for fixing this bug, I slightly modified the failover code 
to make it work as specified in the 
[documentation|https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#Architecture].
 Previously, when failing *+back+* from NN2 back to NN1, the code blindly 
fences NN2 even if NN2 has successfully becomes standby. Since the fencing 
functionality is for making sure that we have at most one active NameNode 
anytime, killing NN2 in this case is unnecessary. Moreover, killing a running 
NameNode will reduce the robustness of the cluster. After this patch, the code 
fences NN2 +*only when NN2 fails to become standby*+. In this way, we still 
make sure that there will be at most one active NameNode any time.

> 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_001-019 
> 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_001-019'
>  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_001-019)
>  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 

[jira] [Commented] (HDFS-15235) Transient network failure during NameNode failover kills the NameNode

2020-03-31 Thread Wei-Chiu Chuang (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-15235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17072102#comment-17072102
 ] 

Wei-Chiu Chuang commented on HDFS-15235:


I've seen some variant of this issue where NNs kept bouncing back and forth. 
However, I don't think it was caused by network partitioning, rather, the sbnn 
didn't respond because of JVM GC pause (or something that prevented NN from 
sending response back)

> 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_001-019 
> 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_001-019'
>  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_001-019)
>  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 3 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 
> 

[jira] [Commented] (HDFS-15235) Transient network failure during NameNode failover kills the NameNode

2020-03-31 Thread YCozy (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-15235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17071727#comment-17071727
 ] 

YCozy commented on HDFS-15235:
--

Hello [~weichiu], would you please help review the patch? Thanks!

> 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_001-019 
> 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_001-019'
>  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_001-019)
>  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 3 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 

[jira] [Commented] (HDFS-15235) Transient network failure during NameNode failover kills the NameNode

2020-03-30 Thread Jira


[ 
https://issues.apache.org/jira/browse/HDFS-15235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17071027#comment-17071027
 ] 

Íñigo Goiri commented on HDFS-15235:


This is very old logic added in HADOOP-7938 more than 8 years ago.
I've touched this code a little but not super experience either.

> 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_001-019 
> 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_001-019'
>  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_001-019)
>  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 3 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 
> 

[jira] [Commented] (HDFS-15235) Transient network failure during NameNode failover kills the NameNode

2020-03-30 Thread Ayush Saxena (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-15235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17071021#comment-17071021
 ] 

Ayush Saxena commented on HDFS-15235:
-

Seems little critical, Fencing if gone wrong can lead to split brain scenario. 
Better somebody with more experience in the area can help.
[~weichiu] [~elgoiri] can you help check here?

> 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_001-019 
> 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_001-019'
>  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_001-019)
>  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 3 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 
> 

[jira] [Commented] (HDFS-15235) Transient network failure during NameNode failover kills the NameNode

2020-03-30 Thread YCozy (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-15235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17071006#comment-17071006
 ] 

YCozy commented on HDFS-15235:
--

Hello [~ayushtkn], I've made the title more accurate. Could you please help 
review the patch? Thanks!

> 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_001-019 
> 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_001-019'
>  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_001-019)
>  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 3 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 

[jira] [Commented] (HDFS-15235) Transient network failure during NameNode failover kills the NameNode

2020-03-28 Thread YCozy (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-15235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17069462#comment-17069462
 ] 

YCozy commented on HDFS-15235:
--

Also, NN2 shouldn't be killed because the fencing should be invoked only when 
we can't make the NN standby, according to the following sentence in the 
[documentation|https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#Architecture]:
{quote}During a failover, if it cannot be verified that the previous Active 
node has relinquished its Active state, the fencing process is responsible for 
cutting off the previous Active’s access to the shared edits storage.
{quote}

> 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_001-019 
> 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_001-019'
>  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_001-019)
>  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 3 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; : 
> 

[jira] [Commented] (HDFS-15235) Transient network failure during NameNode failover kills the NameNode

2020-03-28 Thread YCozy (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-15235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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_001-019 
> 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_001-019'
>  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_001-019)
>  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 3 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 
>