|
@@ -0,0 +1,180 @@
|
|
|
+/**
|
|
|
+ * 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.ha;
|
|
|
+
|
|
|
+import static org.junit.Assert.*;
|
|
|
+
|
|
|
+import java.io.File;
|
|
|
+import java.io.IOException;
|
|
|
+import java.net.URI;
|
|
|
+import java.util.Collections;
|
|
|
+import java.util.List;
|
|
|
+
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
+import org.junit.Test;
|
|
|
+
|
|
|
+import com.google.common.base.Joiner;
|
|
|
+import com.google.common.collect.Lists;
|
|
|
+
|
|
|
+/**
|
|
|
+ * Test cases for the handling of edit logs during failover
|
|
|
+ * and startup of the standby node.
|
|
|
+ */
|
|
|
+public class TestEditLogsDuringFailover {
|
|
|
+ private static final Log LOG =
|
|
|
+ LogFactory.getLog(TestEditLogsDuringFailover.class);
|
|
|
+ private static final int NUM_DIRS_IN_LOG = 5;
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testStartup() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .nnTopology(MiniDFSNNTopology.simpleHATopology())
|
|
|
+ .numDataNodes(0)
|
|
|
+ .build();
|
|
|
+ try {
|
|
|
+ // During HA startup, both nodes should be in
|
|
|
+ // standby and we shouldn't have any edits files
|
|
|
+ // in any edits directory!
|
|
|
+ List<URI> allDirs = Lists.newArrayList();
|
|
|
+ allDirs.addAll(cluster.getNameDirs(0));
|
|
|
+ allDirs.addAll(cluster.getNameDirs(1));
|
|
|
+ allDirs.add(cluster.getSharedEditsDir(0, 1));
|
|
|
+ assertNoEditFiles(allDirs);
|
|
|
+
|
|
|
+ // Set the first NN to active, make sure it creates edits
|
|
|
+ // in its own dirs and the shared dir. The standby
|
|
|
+ // should still have no edits!
|
|
|
+ cluster.getNameNode(0).getRpcServer().transitionToActive();
|
|
|
+
|
|
|
+ assertEditFiles(cluster.getNameDirs(0),
|
|
|
+ NNStorage.getInProgressEditsFileName(1));
|
|
|
+ assertEditFiles(
|
|
|
+ Collections.singletonList(cluster.getSharedEditsDir(0, 1)),
|
|
|
+ NNStorage.getInProgressEditsFileName(1));
|
|
|
+ assertNoEditFiles(cluster.getNameDirs(1));
|
|
|
+
|
|
|
+ cluster.getNameNode(0).getRpcServer().mkdirs("/test",
|
|
|
+ FsPermission.createImmutable((short)0755), true);
|
|
|
+
|
|
|
+ // Restarting the standby should not finalize any edits files
|
|
|
+ // in the shared directory when it starts up!
|
|
|
+ cluster.restartNameNode(1);
|
|
|
+
|
|
|
+ assertEditFiles(cluster.getNameDirs(0),
|
|
|
+ NNStorage.getInProgressEditsFileName(1));
|
|
|
+ assertEditFiles(
|
|
|
+ Collections.singletonList(cluster.getSharedEditsDir(0, 1)),
|
|
|
+ NNStorage.getInProgressEditsFileName(1));
|
|
|
+ assertNoEditFiles(cluster.getNameDirs(1));
|
|
|
+
|
|
|
+ // Additionally it should not have applied any in-progress logs
|
|
|
+ // at start-up -- otherwise, it would have read half-way into
|
|
|
+ // the current log segment, and on the next roll, it would have to
|
|
|
+ // either replay starting in the middle of the segment (not allowed)
|
|
|
+ // or double-replay the edits (incorrect).
|
|
|
+ assertNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1), "/test", true));
|
|
|
+
|
|
|
+ cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
|
|
|
+ FsPermission.createImmutable((short)0755), true);
|
|
|
+
|
|
|
+ // If we restart NN0, it'll come back as standby, and we can
|
|
|
+ // transition NN1 to active and make sure it reads edits correctly at this point.
|
|
|
+ cluster.restartNameNode(0);
|
|
|
+ cluster.getNameNode(1).getRpcServer().transitionToActive();
|
|
|
+
|
|
|
+ // NN1 should have both the edits that came before its restart, and the edits that
|
|
|
+ // came after its restart.
|
|
|
+ assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1), "/test", true));
|
|
|
+ assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1), "/test2", true));
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testFailoverFinalizesAndReadsInProgress() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .nnTopology(MiniDFSNNTopology.simpleHATopology())
|
|
|
+ .numDataNodes(0)
|
|
|
+ .build();
|
|
|
+ try {
|
|
|
+ // Create a fake in-progress edit-log in the shared directory
|
|
|
+ URI sharedUri = cluster.getSharedEditsDir(0, 1);
|
|
|
+ File sharedDir = new File(sharedUri.getPath(), "current");
|
|
|
+ FSImageTestUtil.createAbortedLogWithMkdirs(sharedDir, NUM_DIRS_IN_LOG);
|
|
|
+ assertEditFiles(Collections.singletonList(sharedUri),
|
|
|
+ NNStorage.getInProgressEditsFileName(1));
|
|
|
+
|
|
|
+ // Transition one of the NNs to active
|
|
|
+ cluster.getNameNode(0).getRpcServer().transitionToActive();
|
|
|
+
|
|
|
+ // In the transition to active, it should have read the log -- and
|
|
|
+ // hence see one of the dirs we made in the fake log.
|
|
|
+ String testPath = "/dir" + NUM_DIRS_IN_LOG;
|
|
|
+ assertNotNull(cluster.getNameNode(0).getRpcServer().getFileInfo(testPath));
|
|
|
+
|
|
|
+ // It also should have finalized that log in the shared directory and started
|
|
|
+ // writing to a new one at the next txid.
|
|
|
+ assertEditFiles(Collections.singletonList(sharedUri),
|
|
|
+ NNStorage.getFinalizedEditsFileName(1, NUM_DIRS_IN_LOG + 1),
|
|
|
+ NNStorage.getInProgressEditsFileName(NUM_DIRS_IN_LOG + 2));
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Check that no edits files are present in the given storage dirs.
|
|
|
+ */
|
|
|
+ private void assertNoEditFiles(Iterable<URI> dirs) throws IOException {
|
|
|
+ assertEditFiles(dirs, new String[]{});
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Check that the given list of edits files are present in the given storage
|
|
|
+ * dirs.
|
|
|
+ */
|
|
|
+ private void assertEditFiles(Iterable<URI> dirs, String ... files)
|
|
|
+ throws IOException {
|
|
|
+ for (URI u : dirs) {
|
|
|
+ File editDirRoot = new File(u.getPath());
|
|
|
+ File editDir = new File(editDirRoot, "current");
|
|
|
+ GenericTestUtils.assertExists(editDir);
|
|
|
+ if (files.length == 0) {
|
|
|
+ LOG.info("Checking no edit files exist in " + editDir);
|
|
|
+ } else {
|
|
|
+ LOG.info("Checking for following edit files in " + editDir
|
|
|
+ + ": " + Joiner.on(",").join(files));
|
|
|
+ }
|
|
|
+
|
|
|
+ GenericTestUtils.assertGlobEquals(editDir, "edits_.*", files);
|
|
|
+ }
|
|
|
+ }
|
|
|
+}
|