Просмотр исходного кода

HDFS-10536. Standby NN can not trigger log roll after EditLogTailer thread failed 3 times in EditLogTailer.triggerActiveLogRoll method (Contributed by XingFeng Shen

Vinayakumar B 9 лет назад
Родитель
Сommit
73615a789d

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java

@@ -416,11 +416,11 @@ public class EditLogTailer {
     protected abstract T doWork() throws IOException;
 
     public T call() throws IOException {
+      // reset the loop count on success
+      nnLoopCount = 0;
       while ((cachedActiveProxy = getActiveNodeProxy()) != null) {
         try {
           T ret = doWork();
-          // reset the loop count on success
-          nnLoopCount = 0;
           return ret;
         } catch (RemoteException e) {
           Throwable cause = e.unwrapRemoteException(StandbyException.class);

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java

@@ -18,12 +18,14 @@
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -180,6 +182,50 @@ public class TestEditLogTailer {
       cluster.shutdown();
     }
   }
+
+  /*
+    1. when all NN become standby nn, standby NN execute to roll log,
+    it will be failed.
+    2. when one NN become active, standby NN roll log success.
+   */
+  @Test
+  public void testTriggersLogRollsForAllStandbyNN() throws Exception {
+    Configuration conf = getConf();
+    // Roll every 1s
+    conf.setInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_KEY, 100);
+
+    // Have to specify IPC ports so the NNs can talk to each other.
+    MiniDFSNNTopology topology = new MiniDFSNNTopology()
+        .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
+            .addNN(new MiniDFSNNTopology.NNConf("nn1")
+                .setIpcPort(ServerSocketUtil.getPort(0, 100)))
+            .addNN(new MiniDFSNNTopology.NNConf("nn2")
+                .setIpcPort(ServerSocketUtil.getPort(0, 100)))
+            .addNN(new MiniDFSNNTopology.NNConf("nn3")
+                .setIpcPort(ServerSocketUtil.getPort(0, 100))));
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(topology)
+        .numDataNodes(0)
+        .build();
+    try {
+      cluster.transitionToStandby(0);
+      cluster.transitionToStandby(1);
+      cluster.transitionToStandby(2);
+      try {
+        waitForLogRollInSharedDir(cluster, 3);
+        fail("After all NN become Standby state, Standby NN should roll log, " +
+            "but it will be failed");
+      } catch (TimeoutException ignore) {
+      }
+      cluster.transitionToActive(0);
+      waitForLogRollInSharedDir(cluster, 3);
+    } finally {
+      cluster.shutdown();
+    }
+  }
   
   private static String getDirPath(int suffix) {
     return DIR_PREFIX + suffix;