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

This commit is contained in:
Vinayakumar B 2016-06-26 15:45:32 +05:30
parent d328e66706
commit 73615a789d
2 changed files with 48 additions and 2 deletions

View File

@ -416,11 +416,11 @@ private abstract class MultipleNameNodeProxy<T> implements Callable<T> {
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);

View File

@ -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;
@ -181,6 +183,50 @@ private static void testStandbyTriggersLogRolls(int activeIndex)
}
}
/*
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;
}