|
@@ -27,6 +27,7 @@ import static org.mockito.Mockito.doAnswer;
|
|
import static org.mockito.Mockito.spy;
|
|
import static org.mockito.Mockito.spy;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.util.Arrays;
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
|
|
|
|
@@ -37,6 +38,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.HAUtil;
|
|
import org.apache.hadoop.hdfs.HAUtil;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
|
+import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
|
|
|
|
+import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster.Builder;
|
|
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
|
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
|
@@ -48,23 +51,49 @@ import org.apache.hadoop.util.ExitUtil.ExitException;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
+import org.junit.runner.RunWith;
|
|
|
|
+import org.junit.runners.Parameterized;
|
|
|
|
+import org.junit.runners.Parameterized.Parameters;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
import org.mockito.stubbing.Answer;
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
|
|
import com.google.common.collect.ImmutableList;
|
|
import com.google.common.collect.ImmutableList;
|
|
|
|
|
|
|
|
+@RunWith(Parameterized.class)
|
|
public class TestFailureToReadEdits {
|
|
public class TestFailureToReadEdits {
|
|
|
|
|
|
private static final String TEST_DIR1 = "/test1";
|
|
private static final String TEST_DIR1 = "/test1";
|
|
private static final String TEST_DIR2 = "/test2";
|
|
private static final String TEST_DIR2 = "/test2";
|
|
private static final String TEST_DIR3 = "/test3";
|
|
private static final String TEST_DIR3 = "/test3";
|
|
|
|
|
|
|
|
+ private final TestType clusterType;
|
|
private Configuration conf;
|
|
private Configuration conf;
|
|
private MiniDFSCluster cluster;
|
|
private MiniDFSCluster cluster;
|
|
|
|
+ private MiniQJMHACluster miniQjmHaCluster; // for QJM case only
|
|
private NameNode nn0;
|
|
private NameNode nn0;
|
|
private NameNode nn1;
|
|
private NameNode nn1;
|
|
private FileSystem fs;
|
|
private FileSystem fs;
|
|
|
|
|
|
|
|
+ private static enum TestType {
|
|
|
|
+ SHARED_DIR_HA,
|
|
|
|
+ QJM_HA;
|
|
|
|
+ };
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Run this suite of tests both for QJM-based HA and for file-based
|
|
|
|
+ * HA.
|
|
|
|
+ */
|
|
|
|
+ @Parameters
|
|
|
|
+ public static Iterable<Object[]> data() {
|
|
|
|
+ return Arrays.asList(new Object[][] {
|
|
|
|
+ { TestType.SHARED_DIR_HA },
|
|
|
|
+ { TestType.QJM_HA } });
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public TestFailureToReadEdits(TestType clusterType) {
|
|
|
|
+ this.clusterType = clusterType;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Before
|
|
@Before
|
|
public void setUpCluster() throws Exception {
|
|
public void setUpCluster() throws Exception {
|
|
conf = new Configuration();
|
|
conf = new Configuration();
|
|
@@ -74,16 +103,19 @@ public class TestFailureToReadEdits {
|
|
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|
|
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|
|
HAUtil.setAllowStandbyReads(conf, true);
|
|
HAUtil.setAllowStandbyReads(conf, true);
|
|
|
|
|
|
- MiniDFSNNTopology topology = new MiniDFSNNTopology()
|
|
|
|
- .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
|
|
|
|
- .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10041))
|
|
|
|
- .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10042)));
|
|
|
|
- cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
- .nnTopology(topology)
|
|
|
|
- .numDataNodes(0)
|
|
|
|
- .checkExitOnShutdown(false)
|
|
|
|
- .build();
|
|
|
|
-
|
|
|
|
|
|
+ if (clusterType == TestType.SHARED_DIR_HA) {
|
|
|
|
+ MiniDFSNNTopology topology = MiniQJMHACluster.createDefaultTopology();
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
+ .nnTopology(topology)
|
|
|
|
+ .numDataNodes(0)
|
|
|
|
+ .checkExitOnShutdown(false)
|
|
|
|
+ .build();
|
|
|
|
+ } else {
|
|
|
|
+ Builder builder = new MiniQJMHACluster.Builder(conf);
|
|
|
|
+ builder.getDfsBuilder().numDataNodes(0).checkExitOnShutdown(false);
|
|
|
|
+ miniQjmHaCluster = builder.build();
|
|
|
|
+ cluster = miniQjmHaCluster.getDfsCluster();
|
|
|
|
+ }
|
|
cluster.waitActive();
|
|
cluster.waitActive();
|
|
|
|
|
|
nn0 = cluster.getNameNode(0);
|
|
nn0 = cluster.getNameNode(0);
|
|
@@ -99,8 +131,14 @@ public class TestFailureToReadEdits {
|
|
fs.close();
|
|
fs.close();
|
|
}
|
|
}
|
|
|
|
|
|
- if (cluster != null) {
|
|
|
|
- cluster.shutdown();
|
|
|
|
|
|
+ if (clusterType == TestType.SHARED_DIR_HA) {
|
|
|
|
+ if (cluster != null) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ if (miniQjmHaCluster != null) {
|
|
|
|
+ miniQjmHaCluster.shutdown();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -259,13 +297,10 @@ public class TestFailureToReadEdits {
|
|
}
|
|
}
|
|
|
|
|
|
private LimitedEditLogAnswer causeFailureOnEditLogRead() throws IOException {
|
|
private LimitedEditLogAnswer causeFailureOnEditLogRead() throws IOException {
|
|
- FSEditLog spyEditLog = spy(nn1.getNamesystem().getEditLogTailer()
|
|
|
|
- .getEditLog());
|
|
|
|
|
|
+ FSEditLog spyEditLog = NameNodeAdapter.spyOnEditLog(nn1);
|
|
LimitedEditLogAnswer answer = new LimitedEditLogAnswer();
|
|
LimitedEditLogAnswer answer = new LimitedEditLogAnswer();
|
|
doAnswer(answer).when(spyEditLog).selectInputStreams(
|
|
doAnswer(answer).when(spyEditLog).selectInputStreams(
|
|
anyLong(), anyLong(), (MetaRecoveryContext)anyObject(), anyBoolean());
|
|
anyLong(), anyLong(), (MetaRecoveryContext)anyObject(), anyBoolean());
|
|
- nn1.getNamesystem().getEditLogTailer().setEditLog(spyEditLog);
|
|
|
|
-
|
|
|
|
return answer;
|
|
return answer;
|
|
}
|
|
}
|
|
|
|
|