HBASE-5733 AssignmentManager#processDeadServersAndRegionsInTransition can fail with NPE (Uma Maheswara Rao G)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1327364 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2012-04-18 03:15:24 +00:00
parent 34d88b8e23
commit 8d84537c19
2 changed files with 53 additions and 3 deletions

View File

@ -402,6 +402,13 @@ public class AssignmentManager extends ZooKeeperListener {
throws KeeperException, IOException, InterruptedException {
List<String> nodes = ZKUtil.listChildrenAndWatchForNewChildren(watcher,
watcher.assignmentZNode);
if (nodes == null) {
String errorMessage = "Failed to get the children from ZK";
master.abort(errorMessage, new IOException(errorMessage));
return;
}
// Run through all regions. If they are not assigned and not in RIT, then
// its a clean cluster startup, else its a failover.
for (Map.Entry<HRegionInfo, ServerName> e: this.regions.entrySet()) {

View File

@ -17,8 +17,10 @@
*/
package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
@ -27,6 +29,7 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -39,10 +42,10 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
@ -54,10 +57,12 @@ import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.junit.After;
import org.junit.AfterClass;
@ -477,6 +482,39 @@ public class TestAssignmentManager {
}
}
/**
* Tests the processDeadServersAndRegionsInTransition should not fail with NPE
* when it failed to get the children. Let's abort the system in this
* situation
* @throws ServiceException
*/
@Test(timeout = 5000)
public void testProcessDeadServersAndRegionsInTransitionShouldNotFailWithNPE()
throws IOException, KeeperException, InterruptedException, ServiceException {
final RecoverableZooKeeper recoverableZk = Mockito
.mock(RecoverableZooKeeper.class);
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
this.server, this.serverManager);
Watcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest",
null) {
public RecoverableZooKeeper getRecoverableZooKeeper() {
return recoverableZk;
}
};
((ZooKeeperWatcher) zkw).registerListener(am);
Mockito.doThrow(new InterruptedException()).when(recoverableZk)
.getChildren("/hbase/unassigned", zkw);
am.setWatcher((ZooKeeperWatcher) zkw);
try {
am.processDeadServersAndRegionsInTransition();
fail("Expected to abort");
} catch (NullPointerException e) {
fail("Should not throw NPE");
} catch (RuntimeException e) {
assertEquals("Aborted", e.getLocalizedMessage());
}
}
/**
* Creates a new ephemeral node in the SPLITTING state for the specified region.
* Create it ephemeral in case regionserver dies mid-split.
@ -610,7 +648,12 @@ public class TestAssignmentManager {
while (this.gate.get()) Threads.sleep(1);
super.processRegionsInTransition(data, regionInfo, deadServers, expectedVersion);
}
/** reset the watcher */
void setWatcher(ZooKeeperWatcher watcher) {
this.watcher = watcher;
}
/**
* @return ExecutorService used by this instance.
*/