HDFS-3432. TestDFSZKFailoverController tries to fail over too early. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3042@1339434 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2012-05-17 00:36:13 +00:00
parent e14e10fb86
commit 0600e834be
5 changed files with 54 additions and 9 deletions

View File

@ -22,6 +22,7 @@
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -43,7 +44,8 @@
* Classes which need callbacks should implement the {@link Callback} * Classes which need callbacks should implement the {@link Callback}
* interface. * interface.
*/ */
class HealthMonitor { @InterfaceAudience.Private
public class HealthMonitor {
private static final Log LOG = LogFactory.getLog( private static final Log LOG = LogFactory.getLog(
HealthMonitor.class); HealthMonitor.class);
@ -75,7 +77,8 @@ class HealthMonitor {
private HAServiceStatus lastServiceState = new HAServiceStatus( private HAServiceStatus lastServiceState = new HAServiceStatus(
HAServiceState.INITIALIZING); HAServiceState.INITIALIZING);
enum State { @InterfaceAudience.Private
public enum State {
/** /**
* The health monitor is still starting up. * The health monitor is still starting up.
*/ */

View File

@ -170,11 +170,7 @@ public void waitForHAState(int idx, HAServiceState state)
*/ */
public void waitForHealthState(int idx, State state) public void waitForHealthState(int idx, State state)
throws Exception { throws Exception {
ZKFailoverController zkfc = thrs[idx].zkfc; ZKFCTestUtil.waitForHealthState(thrs[idx].zkfc, state, ctx);
while (zkfc.getLastHealthState() != state) {
ctx.checkException();
Thread.sleep(50);
}
} }
/** /**

View File

@ -0,0 +1,34 @@
/**
* 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.ha;
import org.apache.hadoop.test.MultithreadedTestUtil;
public class ZKFCTestUtil {
public static void waitForHealthState(ZKFailoverController zkfc,
HealthMonitor.State state,
MultithreadedTestUtil.TestContext ctx) throws Exception {
while (zkfc.getLastHealthState() != state) {
if (ctx != null) {
ctx.checkException();
}
Thread.sleep(50);
}
}
}

View File

@ -15,3 +15,5 @@ HDFS-3261. TestHASafeMode fails on HDFS-3042 branch (todd)
HDFS-3159. Document NN auto-failover setup and configuration (todd) HDFS-3159. Document NN auto-failover setup and configuration (todd)
HDFS-3412. Fix findbugs warnings in auto-HA branch (todd) HDFS-3412. Fix findbugs warnings in auto-HA branch (todd)
HDFS-3432. TestDFSZKFailoverController tries to fail over too early (todd)

View File

@ -26,6 +26,8 @@
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.ha.ClientBaseWithFixes; import org.apache.hadoop.ha.ClientBaseWithFixes;
import org.apache.hadoop.ha.HealthMonitor;
import org.apache.hadoop.ha.ZKFCTestUtil;
import org.apache.hadoop.ha.ZKFailoverController; import org.apache.hadoop.ha.ZKFailoverController;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer; import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
@ -91,6 +93,12 @@ public void setup() throws Exception {
ctx.addThread(thr2 = new ZKFCThread(ctx, 1)); ctx.addThread(thr2 = new ZKFCThread(ctx, 1));
thr2.start(); thr2.start();
// Wait for the ZKFCs to fully start up
ZKFCTestUtil.waitForHealthState(thr1.zkfc,
HealthMonitor.State.SERVICE_HEALTHY, ctx);
ZKFCTestUtil.waitForHealthState(thr2.zkfc,
HealthMonitor.State.SERVICE_HEALTHY, ctx);
fs = HATestUtil.configureFailoverFs(cluster, conf); fs = HATestUtil.configureFailoverFs(cluster, conf);
} }
@ -160,10 +168,12 @@ public void testManualFailover() throws Exception {
public void testManualFailoverWithDFSHAAdmin() throws Exception { public void testManualFailoverWithDFSHAAdmin() throws Exception {
DFSHAAdmin tool = new DFSHAAdmin(); DFSHAAdmin tool = new DFSHAAdmin();
tool.setConf(conf); tool.setConf(conf);
tool.run(new String[]{"-failover", "nn1", "nn2"}); assertEquals(0,
tool.run(new String[]{"-failover", "nn1", "nn2"}));
waitForHAState(0, HAServiceState.STANDBY); waitForHAState(0, HAServiceState.STANDBY);
waitForHAState(1, HAServiceState.ACTIVE); waitForHAState(1, HAServiceState.ACTIVE);
tool.run(new String[]{"-failover", "nn2", "nn1"}); assertEquals(0,
tool.run(new String[]{"-failover", "nn2", "nn1"}));
waitForHAState(0, HAServiceState.ACTIVE); waitForHAState(0, HAServiceState.ACTIVE);
waitForHAState(1, HAServiceState.STANDBY); waitForHAState(1, HAServiceState.STANDBY);
} }