HDFS-3026. HA: Handle failure during HA state transition. Contributed by Aaron T. Myers.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1337031 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2012-05-11 06:02:33 +00:00
parent 6795b5ad8f
commit 97a8118315
3 changed files with 149 additions and 18 deletions

View File

@ -532,6 +532,8 @@ Release 2.0.0 - UNRELEASED
necessarily a BlockInfoUnderConstruction, so do not cast it in
FSNamesystem.recoverLeaseInternal(..). (szetszwo)
HDFS-3026. HA: Handle failure during HA state transition. (atm)
BREAKDOWN OF HDFS-1623 SUBTASKS
HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)

View File

@ -206,6 +206,7 @@ public class NameNode {
private final boolean haEnabled;
private final HAContext haContext;
protected boolean allowStaleStandbyReads;
private Runtime runtime = Runtime.getRuntime();
/** httpServer */
@ -481,11 +482,16 @@ public class NameNode {
}
private void startTrashEmptier(Configuration conf) throws IOException {
long trashInterval
= conf.getLong(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY,
long trashInterval = conf.getLong(
CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY,
CommonConfigurationKeys.FS_TRASH_INTERVAL_DEFAULT);
if(trashInterval == 0)
if (trashInterval == 0) {
return;
} else if (trashInterval < 0) {
throw new IOException("Cannot start tresh emptier with negative interval."
+ " Set " + CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY + " to a"
+ " positive value.");
}
this.emptier = new Thread(new Trash(conf).getEmptier(), "Trash Emptier");
this.emptier.setDaemon(true);
this.emptier.start();
@ -1236,13 +1242,36 @@ public class NameNode {
return state.getServiceState();
}
@VisibleForTesting
public synchronized void setRuntimeForTesting(Runtime runtime) {
this.runtime = runtime;
}
/**
* Class used as expose {@link NameNode} as context to {@link HAState}
* Shutdown the NN immediately in an ungraceful way. Used when it would be
* unsafe for the NN to continue operating, e.g. during a failed HA state
* transition.
*
* TODO(HA):
* When entering and exiting state, on failing to start services,
* appropriate action is needed todo either shutdown the node or recover
* from failure.
* @param t exception which warrants the shutdown. Printed to the NN log
* before exit.
* @throws ServiceFailedException thrown only for testing.
*/
private synchronized void doImmediateShutdown(Throwable t)
throws ServiceFailedException {
String message = "Error encountered requiring NN shutdown. " +
"Shutting down immediately.";
try {
LOG.fatal(message, t);
} catch (Throwable ignored) {
// This is unlikely to happen, but there's nothing we can do if it does.
}
runtime.exit(1);
// This code is only reached during testing, when runtime is stubbed out.
throw new ServiceFailedException(message, t);
}
/**
* Class used to expose {@link NameNode} as context to {@link HAState}
*/
protected class NameNodeHAContext implements HAContext {
@Override
@ -1257,33 +1286,53 @@ public class NameNode {
@Override
public void startActiveServices() throws IOException {
try {
namesystem.startActiveServices();
startTrashEmptier(conf);
} catch (Throwable t) {
doImmediateShutdown(t);
}
}
@Override
public void stopActiveServices() throws IOException {
try {
if (namesystem != null) {
namesystem.stopActiveServices();
}
stopTrashEmptier();
} catch (Throwable t) {
doImmediateShutdown(t);
}
}
@Override
public void startStandbyServices() throws IOException {
try {
namesystem.startStandbyServices(conf);
} catch (Throwable t) {
doImmediateShutdown(t);
}
}
@Override
public void prepareToStopStandbyServices() throws ServiceFailedException {
try {
namesystem.prepareToStopStandbyServices();
} catch (Throwable t) {
doImmediateShutdown(t);
}
}
@Override
public void stopStandbyServices() throws IOException {
try {
if (namesystem != null) {
namesystem.stopStandbyServices();
}
} catch (Throwable t) {
doImmediateShutdown(t);
}
}
@Override

View File

@ -0,0 +1,80 @@
/**
* 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.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.ha.ServiceFailedException;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.junit.Test;
/**
* Tests to verify the behavior of failing to fully start transition HA states.
*/
public class TestStateTransitionFailure {
public static final Log LOG = LogFactory.getLog(TestStateTransitionFailure.class);
/**
* Ensure that a failure to fully transition to the active state causes a
* shutdown of the NameNode.
*/
@Test
public void testFailureToTransitionCausesShutdown() throws IOException {
MiniDFSCluster cluster = null;
try {
Configuration conf = new Configuration();
// Set an illegal value for the trash emptier interval. This will cause
// the NN to fail to transition to the active state.
conf.setLong(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, -1);
cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(MiniDFSNNTopology.simpleHATopology())
.numDataNodes(0)
.build();
cluster.waitActive();
Runtime mockRuntime = mock(Runtime.class);
cluster.getNameNode(0).setRuntimeForTesting(mockRuntime);
verify(mockRuntime, times(0)).exit(anyInt());
try {
cluster.transitionToActive(0);
fail("Transitioned to active but should not have been able to.");
} catch (ServiceFailedException sfe) {
assertExceptionContains("Error encountered requiring NN shutdown. " +
"Shutting down immediately.", sfe);
LOG.info("got expected exception", sfe);
}
verify(mockRuntime, times(1)).exit(anyInt());
} finally {
if (cluster != null) {
cluster.shutdown();
}
}
}
}