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

ASF GitHub Bot commented on HDFS-16689:
---------------------------------------

ZanderXu commented on code in PR #4744:
URL: https://github.com/apache/hadoop/pull/4744#discussion_r963522811


##########
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHAWithInProgressTail.java:
##########
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
+import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
+import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
+import org.apache.hadoop.hdfs.qjournal.client.SpyQJournalUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerFaultInjector;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static 
org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getFileInfo;
+import static org.junit.Assert.assertNotNull;
+
+public class TestHAWithInProgressTail {
+  private MiniQJMHACluster qjmhaCluster;
+  private MiniDFSCluster cluster;
+  private MiniJournalCluster jnCluster;
+  private NameNode nn0;
+  private NameNode nn1;
+
+  @Before
+  public void startUp() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
+    conf.setInt(DFSConfigKeys.DFS_QJOURNAL_SELECT_INPUT_STREAMS_TIMEOUT_KEY, 
500);
+    HAUtil.setAllowStandbyReads(conf, true);
+    qjmhaCluster = new MiniQJMHACluster.Builder(conf).build();
+    cluster = qjmhaCluster.getDfsCluster();
+    jnCluster = qjmhaCluster.getJournalCluster();
+
+    // Get NameNode from cluster to future manual control
+    nn0 = cluster.getNameNode(0);
+    nn1 = cluster.getNameNode(1);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (qjmhaCluster != null) {
+      qjmhaCluster.shutdown();
+    }
+  }
+
+
+  /**
+   * Test that Standby Node tails multiple segments while catching up
+   * during the transition to Active.
+   */
+  @Test
+  public void testFailoverWithAbnormalJN() throws Exception {
+    cluster.transitionToActive(0);
+    cluster.waitActive(0);
+
+    BlockManagerFaultInjector.instance = new BlockManagerFaultInjector() {
+      @Override
+      public void mockJNStreams() throws IOException {
+        spyOnJASjournal();
+      }
+    };

Review Comment:
   @xkrogen Sir, I have updated this patch, please help me review it again. 
Thanks





> Standby NameNode crashes when transitioning to Active with in-progress tailer
> -----------------------------------------------------------------------------
>
>                 Key: HDFS-16689
>                 URL: https://issues.apache.org/jira/browse/HDFS-16689
>             Project: Hadoop HDFS
>          Issue Type: Bug
>            Reporter: ZanderXu
>            Assignee: ZanderXu
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 3.4.0
>
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> Standby NameNode crashes when transitioning to Active with a in-progress 
> tailer. And the error message like blew:
> {code:java}
> Caused by: java.lang.IllegalStateException: Cannot start writing at txid X 
> when there is a stream available for read: ByteStringEditLog[X, Y], 
> ByteStringEditLog[X, 0]
>       at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.openForWrite(FSEditLog.java:344)
>       at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.openForWrite(FSEditLogAsync.java:113)
>       at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startActiveServices(FSNamesystem.java:1423)
>       at 
> org.apache.hadoop.hdfs.server.namenode.NameNode$NameNodeHAContext.startActiveServices(NameNode.java:2132)
>       ... 36 more
> {code}
> After tracing and found there is a critical bug in 
> *EditlogTailer#catchupDuringFailover()* when 
> *DFS_HA_TAILEDITS_INPROGRESS_KEY* is true. Because *catchupDuringFailover()* 
> try to replay all missed edits from JournalNodes with *onlyDurableTxns=true*. 
> It may cannot replay any edits when they are some abnormal JournalNodes. 
> Reproduce method, suppose:
> - There are 2 namenode, namely NN0 and NN1, and the status of echo namenode 
> is Active, Standby respectively. And there are 3 JournalNodes, namely JN0, 
> JN1 and JN2. 
> - NN0 try to sync 3 edits to JNs with started txid 3, but only successfully 
> synced them to JN1 and JN3. And JN0 is abnormal, such as GC, bad network or 
> restarted.
> - NN1's lastAppliedTxId is 2, and at the moment, we are trying failover 
> active from NN0 to NN1. 
> - NN1 only got two responses from JN0 and JN1 when it try to selecting 
> inputStreams with *fromTxnId=3*  and *onlyDurableTxns=true*, and the count 
> txid of response is 0, 3 respectively. JN2 is abnormal, such as GC,  bad 
> network or restarted.
> - NN1 will cannot replay any Edits with *fromTxnId=3* from JournalNodes 
> because the *maxAllowedTxns* is 0.
> So I think Standby NameNode should *catchupDuringFailover()* with 
> *onlyDurableTxns=false* , so that it can replay all missed edits from 
> JournalNode.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to