HBASE-4540 OpenedRegionHandler is not enforcing atomicity of the operation it is performing . Also fixes HBASE-4539 (ram)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1180553 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
251df1d35f
commit
3d50077db1
|
@ -344,6 +344,7 @@ Release 0.92.0 - Unreleased
|
|||
HBASE-4402 Retaining locality after restart broken
|
||||
HBASE-4482 Race Condition Concerning Eviction in SlabCache (Li Pi)
|
||||
HBASE-4547 TestAdmin failing in 0.92 because .tableinfo not found
|
||||
HBASE-4540 OpenedRegionHandler is not enforcing atomicity of the operation it is performing(Ram)
|
||||
|
||||
TESTS
|
||||
HBASE-4450 test for number of blocks read: to serve as baseline for expected
|
||||
|
|
|
@ -426,7 +426,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
final HRegionInfo regionInfo,
|
||||
final Map<ServerName,List<Pair<HRegionInfo,Result>>> deadServers)
|
||||
throws KeeperException, IOException {
|
||||
RegionTransitionData data = ZKAssign.getData(watcher, encodedRegionName);
|
||||
Stat stat = new Stat();
|
||||
RegionTransitionData data = ZKAssign.getDataAndWatch(watcher,
|
||||
encodedRegionName, stat);
|
||||
if (data == null) return false;
|
||||
HRegionInfo hri = regionInfo;
|
||||
if (hri == null) {
|
||||
|
@ -435,13 +437,14 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
if (p == null) return false;
|
||||
hri = p.getFirst();
|
||||
}
|
||||
processRegionsInTransition(data, hri, deadServers);
|
||||
processRegionsInTransition(data, hri, deadServers, stat.getVersion());
|
||||
return true;
|
||||
}
|
||||
|
||||
void processRegionsInTransition(final RegionTransitionData data,
|
||||
final HRegionInfo regionInfo,
|
||||
final Map<ServerName,List<Pair<HRegionInfo,Result>>> deadServers)
|
||||
final Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers,
|
||||
int expectedVersion)
|
||||
throws KeeperException {
|
||||
String encodedRegionName = regionInfo.getEncodedName();
|
||||
LOG.info("Processing region " + regionInfo.getRegionNameAsString() +
|
||||
|
@ -516,7 +519,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|| regionInfo.isMetaRegion() || regionInfo.isRootRegion())) {
|
||||
forceOffline(regionInfo, data);
|
||||
} else {
|
||||
new OpenedRegionHandler(master, this, regionInfo, sn).process();
|
||||
new OpenedRegionHandler(master, this, regionInfo, sn, expectedVersion)
|
||||
.process();
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -593,8 +597,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* This deals with skipped transitions (we got a CLOSED but didn't see CLOSING
|
||||
* yet).
|
||||
* @param data
|
||||
* @param expectedVersion
|
||||
*/
|
||||
private void handleRegion(final RegionTransitionData data) {
|
||||
private void handleRegion(final RegionTransitionData data, int expectedVersion) {
|
||||
synchronized(regionsInTransition) {
|
||||
if (data == null || data.getOrigin() == null) {
|
||||
LOG.warn("Unexpected NULL input " + data);
|
||||
|
@ -755,7 +760,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
data.getStamp(), data.getOrigin());
|
||||
this.executorService.submit(
|
||||
new OpenedRegionHandler(master, this, regionState.getRegion(),
|
||||
data.getOrigin()));
|
||||
data.getOrigin(), expectedVersion));
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -907,11 +912,12 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
public void nodeCreated(String path) {
|
||||
if(path.startsWith(watcher.assignmentZNode)) {
|
||||
try {
|
||||
RegionTransitionData data = ZKAssign.getData(watcher, path);
|
||||
Stat stat = new Stat();
|
||||
RegionTransitionData data = ZKAssign.getDataAndWatch(watcher, path, stat);
|
||||
if (data == null) {
|
||||
return;
|
||||
}
|
||||
handleRegion(data);
|
||||
handleRegion(data, stat.getVersion());
|
||||
} catch (KeeperException e) {
|
||||
master.abort("Unexpected ZK exception reading unassigned node data", e);
|
||||
}
|
||||
|
@ -934,11 +940,12 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
public void nodeDataChanged(String path) {
|
||||
if(path.startsWith(watcher.assignmentZNode)) {
|
||||
try {
|
||||
RegionTransitionData data = ZKAssign.getData(watcher, path);
|
||||
Stat stat = new Stat();
|
||||
RegionTransitionData data = ZKAssign.getDataAndWatch(watcher, path, stat);
|
||||
if (data == null) {
|
||||
return;
|
||||
}
|
||||
handleRegion(data);
|
||||
handleRegion(data, stat.getVersion());
|
||||
} catch (KeeperException e) {
|
||||
master.abort("Unexpected ZK exception reading unassigned node data", e);
|
||||
}
|
||||
|
|
|
@ -26,7 +26,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -39,6 +41,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
private final HRegionInfo regionInfo;
|
||||
private final ServerName sn;
|
||||
private final OpenedPriority priority;
|
||||
private final int expectedVersion;
|
||||
|
||||
private enum OpenedPriority {
|
||||
ROOT (1),
|
||||
|
@ -56,11 +59,12 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
|
||||
public OpenedRegionHandler(Server server,
|
||||
AssignmentManager assignmentManager, HRegionInfo regionInfo,
|
||||
ServerName sn) {
|
||||
ServerName sn, int expectedVersion) {
|
||||
super(server, EventType.RS_ZK_REGION_OPENED);
|
||||
this.assignmentManager = assignmentManager;
|
||||
this.regionInfo = regionInfo;
|
||||
this.sn = sn;
|
||||
this.expectedVersion = expectedVersion;
|
||||
if(regionInfo.isRootRegion()) {
|
||||
priority = OpenedPriority.ROOT;
|
||||
} else if(regionInfo.isMetaRegion()) {
|
||||
|
@ -91,23 +95,21 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
|
||||
@Override
|
||||
public void process() {
|
||||
debugLog(regionInfo, "Handling OPENED event for " +
|
||||
this.regionInfo.getRegionNameAsString() + " from " + this.sn.toString()
|
||||
+ "; deleting unassigned node");
|
||||
// Remove region from in-memory transition and unassigned node from ZK
|
||||
try {
|
||||
ZKAssign.deleteOpenedNode(server.getZooKeeper(),
|
||||
regionInfo.getEncodedName());
|
||||
} catch (KeeperException e) {
|
||||
server.abort("Error deleting OPENED node in ZK for transition ZK node ("
|
||||
+ regionInfo.getRegionNameAsString() + ")", e);
|
||||
}
|
||||
// Code to defend against case where we get SPLIT before region open
|
||||
// processing completes; temporary till we make SPLITs go via zk -- 0.92.
|
||||
if (this.assignmentManager.isRegionInTransition(regionInfo) != null) {
|
||||
RegionState regionState = this.assignmentManager.isRegionInTransition(regionInfo);
|
||||
if (regionState != null
|
||||
&& regionState.getState().equals(RegionState.State.OPEN)) {
|
||||
if (deleteOpenedNode(expectedVersion)) {
|
||||
// Remove region from in-memory transition and unassigned node from ZK
|
||||
this.assignmentManager.regionOnline(regionInfo, this.sn);
|
||||
debugLog(regionInfo, "region online: "
|
||||
+ regionInfo.getRegionNameAsString() + " on " + this.sn.toString());
|
||||
debugLog(regionInfo, "The master has opened the region " +
|
||||
regionInfo.getRegionNameAsString() + " that was online on " +
|
||||
this.sn.toString());
|
||||
} else {
|
||||
LOG.error("The znode of region " + regionInfo.getRegionNameAsString() +
|
||||
" could not be deleted.");
|
||||
}
|
||||
} else {
|
||||
LOG.warn("Skipping the onlining of " + regionInfo.getRegionNameAsString() +
|
||||
" because regions is NOT in RIT -- presuming this is because it SPLIT");
|
||||
|
@ -118,12 +120,29 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
"Opened region " + regionInfo.getRegionNameAsString() + " but "
|
||||
+ "this table is disabled, triggering close of region");
|
||||
assignmentManager.unassign(regionInfo);
|
||||
} else {
|
||||
debugLog(regionInfo, "Opened region " + regionInfo.getRegionNameAsString() +
|
||||
" on " + this.sn.toString());
|
||||
}
|
||||
}
|
||||
|
||||
private boolean deleteOpenedNode(int expectedVersion) {
|
||||
debugLog(regionInfo, "Handling OPENED event for " +
|
||||
this.regionInfo.getRegionNameAsString() + " from " + this.sn.toString() +
|
||||
"; deleting unassigned node");
|
||||
try {
|
||||
// delete the opened znode only if the version matches.
|
||||
return ZKAssign.deleteNode(server.getZooKeeper(),
|
||||
regionInfo.getEncodedName(), EventType.RS_ZK_REGION_OPENED, expectedVersion);
|
||||
} catch(KeeperException.NoNodeException e){
|
||||
// Getting no node exception here means that already the region has been opened.
|
||||
LOG.warn("The znode of the region " + regionInfo.getRegionNameAsString() +
|
||||
" would have already been deleted");
|
||||
return false;
|
||||
} catch (KeeperException e) {
|
||||
server.abort("Error deleting OPENED node in ZK (" +
|
||||
regionInfo.getRegionNameAsString() + ")", e);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private void debugLog(HRegionInfo region, String string) {
|
||||
if (region.isMetaTable() || region.isRootRegion()) {
|
||||
LOG.info(string);
|
||||
|
|
|
@ -437,9 +437,9 @@ public class ZKAssign {
|
|||
*
|
||||
* <p>Returns false if the node was not in the proper state but did exist.
|
||||
*
|
||||
* <p>This method is used during table disables when a region finishes
|
||||
* successfully closing. This is the Master acknowledging completion
|
||||
* of the specified regions transition to being closed.
|
||||
* <p>This method is used when a region finishes opening/closing.
|
||||
* The Master acknowledges completion
|
||||
* of the specified regions transition to being closed/opened.
|
||||
*
|
||||
* @param zkw zk reference
|
||||
* @param regionName region to be deleted from zk
|
||||
|
@ -449,6 +449,36 @@ public class ZKAssign {
|
|||
*/
|
||||
public static boolean deleteNode(ZooKeeperWatcher zkw, String regionName,
|
||||
EventType expectedState)
|
||||
throws KeeperException, KeeperException.NoNodeException {
|
||||
return deleteNode(zkw, regionName, expectedState, -1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes an existing unassigned node that is in the specified state for the
|
||||
* specified region.
|
||||
*
|
||||
* <p>If a node does not already exist for this region, a
|
||||
* {@link NoNodeException} will be thrown.
|
||||
*
|
||||
* <p>No watcher is set whether this succeeds or not.
|
||||
*
|
||||
* <p>Returns false if the node was not in the proper state but did exist.
|
||||
*
|
||||
* <p>This method is used when a region finishes opening/closing.
|
||||
* The Master acknowledges completion
|
||||
* of the specified regions transition to being closed/opened.
|
||||
*
|
||||
* @param zkw zk reference
|
||||
* @param regionName region to be deleted from zk
|
||||
* @param expectedState state region must be in for delete to complete
|
||||
* @param expectedVersion of the znode that is to be deleted.
|
||||
* If expectedVersion need not be compared while deleting the znode
|
||||
* pass -1
|
||||
* @throws KeeperException if unexpected zookeeper exception
|
||||
* @throws KeeperException.NoNodeException if node does not exist
|
||||
*/
|
||||
public static boolean deleteNode(ZooKeeperWatcher zkw, String regionName,
|
||||
EventType expectedState, int expectedVersion)
|
||||
throws KeeperException, KeeperException.NoNodeException {
|
||||
LOG.debug(zkw.prefix("Deleting existing unassigned " +
|
||||
"node for " + regionName + " that is in expected state " + expectedState));
|
||||
|
@ -467,6 +497,12 @@ public class ZKAssign {
|
|||
" state but node is in " + data.getEventType() + " state"));
|
||||
return false;
|
||||
}
|
||||
if (expectedVersion != -1
|
||||
&& stat.getVersion() != expectedVersion) {
|
||||
LOG.warn("The node we are trying to delete is not the expected one. " +
|
||||
"Got a version mismatch");
|
||||
return false;
|
||||
}
|
||||
if(!ZKUtil.deleteNode(zkw, node, stat.getVersion())) {
|
||||
LOG.warn(zkw.prefix("Attempting to delete " +
|
||||
"unassigned node in " + expectedState +
|
||||
|
@ -820,6 +856,32 @@ public class ZKAssign {
|
|||
return RegionTransitionData.fromBytes(data);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the current data in the unassigned node for the specified region name
|
||||
* or fully-qualified path.
|
||||
*
|
||||
* <p>Returns null if the region does not currently have a node.
|
||||
*
|
||||
* <p>Sets a watch on the node if the node exists.
|
||||
*
|
||||
* @param zkw zk reference
|
||||
* @param pathOrRegionName fully-specified path or region name
|
||||
* @param stat object to populate the version.
|
||||
* @return data for the unassigned node
|
||||
* @throws KeeperException if unexpected zookeeper exception
|
||||
*/
|
||||
public static RegionTransitionData getDataAndWatch(ZooKeeperWatcher zkw,
|
||||
String pathOrRegionName, Stat stat)
|
||||
throws KeeperException {
|
||||
String node = pathOrRegionName.startsWith("/") ?
|
||||
pathOrRegionName : getNodeName(zkw, pathOrRegionName);
|
||||
byte [] data = ZKUtil.getDataAndWatch(zkw, node, stat);
|
||||
if(data == null) {
|
||||
return null;
|
||||
}
|
||||
return RegionTransitionData.fromBytes(data);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the current data in the unassigned node for the specified region name
|
||||
* or fully-qualified path.
|
||||
|
|
|
@ -482,10 +482,33 @@ public class ZKUtil {
|
|||
* @throws KeeperException if unexpected zookeeper exception
|
||||
*/
|
||||
public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode)
|
||||
throws KeeperException {
|
||||
return getDataInternal(zkw, znode, null, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the data at the specified znode and set a watch.
|
||||
*
|
||||
* Returns the data and sets a watch if the node exists. Returns null and no
|
||||
* watch is set if the node does not exist or there is an exception.
|
||||
*
|
||||
* @param zkw zk reference
|
||||
* @param znode path of node
|
||||
* @param stat object to populate the version of the znode
|
||||
* @return data of the specified znode, or null
|
||||
* @throws KeeperException if unexpected zookeeper exception
|
||||
*/
|
||||
public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode,
|
||||
Stat stat) throws KeeperException {
|
||||
return getDataInternal(zkw, znode, stat, true);
|
||||
}
|
||||
|
||||
private static byte[] getDataInternal(ZooKeeperWatcher zkw, String znode, Stat stat,
|
||||
boolean watcherSet)
|
||||
throws KeeperException {
|
||||
try {
|
||||
byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, null);
|
||||
logRetrievedMsg(zkw, znode, data, true);
|
||||
byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
|
||||
logRetrievedMsg(zkw, znode, data, watcherSet);
|
||||
return data;
|
||||
} catch (KeeperException.NoNodeException e) {
|
||||
LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
|
||||
|
|
|
@ -0,0 +1,188 @@
|
|||
/**
|
||||
* 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.hbase.master;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
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.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
|
||||
import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
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.data.Stat;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestOpenedRegionHandler {
|
||||
|
||||
private static final Log LOG = LogFactory
|
||||
.getLog(TestOpenedRegionHandler.class);
|
||||
|
||||
private HBaseTestingUtility TEST_UTIL;
|
||||
private final int NUM_MASTERS = 1;
|
||||
private final int NUM_RS = 1;
|
||||
private Configuration conf;
|
||||
private Configuration resetConf;
|
||||
private ZooKeeperWatcher zkw;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
// Start the cluster
|
||||
log("Starting cluster");
|
||||
conf = HBaseConfiguration.create();
|
||||
resetConf = conf;
|
||||
conf.setInt("hbase.master.assignment.timeoutmonitor.period", 2000);
|
||||
conf.setInt("hbase.master.assignment.timeoutmonitor.timeout", 5000);
|
||||
TEST_UTIL = new HBaseTestingUtility(conf);
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
// Stop the cluster
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
TEST_UTIL = new HBaseTestingUtility(resetConf);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOpenedRegionHandlerOnMasterRestart() throws Exception {
|
||||
String tableName = "testOpenedRegionHandlerOnMasterRestart";
|
||||
MiniHBaseCluster cluster = createRegions(tableName);
|
||||
abortMaster(cluster);
|
||||
|
||||
HRegionServer regionServer = cluster.getRegionServer(0);
|
||||
HRegion region = getRegionBeingServed(cluster, regionServer);
|
||||
|
||||
// forcefully move a region to OPENED state in zk
|
||||
// Create a ZKW to use in the test
|
||||
zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
|
||||
region, regionServer);
|
||||
|
||||
// Start up a new master
|
||||
log("Starting up a new master");
|
||||
cluster.startMaster().getMaster();
|
||||
log("Waiting for master to be ready");
|
||||
cluster.waitForActiveAndReadyMaster();
|
||||
log("Master is ready");
|
||||
|
||||
// Failover should be completed, now wait for no RIT
|
||||
log("Waiting for no more RIT");
|
||||
ZKAssign.blockUntilNoRIT(zkw);
|
||||
}
|
||||
@Test
|
||||
public void testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches()
|
||||
throws Exception {
|
||||
String tableName = "testIfVersionMismatches";
|
||||
MiniHBaseCluster cluster = createRegions(tableName);
|
||||
AssignmentManager am = cluster.getMaster().assignmentManager;
|
||||
abortMaster(cluster);
|
||||
|
||||
HRegionServer regionServer = cluster.getRegionServer(0);
|
||||
HRegion region = getRegionBeingServed(cluster, regionServer);
|
||||
|
||||
cluster.stopRegionServer(0);
|
||||
//create a node with OPENED state
|
||||
zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
|
||||
region, regionServer);
|
||||
Stat stat = new Stat();
|
||||
String nodeName = ZKAssign.getNodeName(zkw, region
|
||||
.getRegionNameAsString());
|
||||
ZKUtil.getDataAndWatch(zkw, nodeName, stat);
|
||||
// use the version for the OpenedRegionHandler
|
||||
OpenedRegionHandler handler = new OpenedRegionHandler(regionServer, am,
|
||||
region.getRegionInfo(), regionServer
|
||||
.getServerName(), stat.getVersion());
|
||||
am.regionsInTransition.put(region
|
||||
.getRegionInfo().getEncodedName(), new RegionState(
|
||||
region.getRegionInfo(), RegionState.State.OPEN,
|
||||
System.currentTimeMillis(), regionServer.getServerName()));
|
||||
|
||||
//Once again overwrite the same znode so that the version changes.
|
||||
ZKAssign.transitionNodeOpened(zkw, region.getRegionInfo(), regionServer
|
||||
.getServerName(), stat.getVersion()+1);
|
||||
|
||||
// try processing the opened region.
|
||||
handler.process();
|
||||
List<String> znodes = ZKUtil.listChildrenAndWatchForNewChildren(zkw,
|
||||
zkw.assignmentZNode);
|
||||
String regionName = znodes.get(0);
|
||||
assertEquals("The region should not be opened successfully.", regionName,
|
||||
region.getRegionInfo().getEncodedName());
|
||||
}
|
||||
private MiniHBaseCluster createRegions(String tableName)
|
||||
throws InterruptedException, ZooKeeperConnectionException, IOException,
|
||||
KeeperException {
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
log("Waiting for active/ready master");
|
||||
cluster.waitForActiveAndReadyMaster();
|
||||
zkw = new ZooKeeperWatcher(conf, "testOpenedRegionHandler", null);
|
||||
|
||||
// Create a table with regions
|
||||
byte[] table = Bytes.toBytes(tableName);
|
||||
byte[] family = Bytes.toBytes("family");
|
||||
TEST_UTIL.createTable(table, family);
|
||||
|
||||
//wait till the regions are online
|
||||
log("Waiting for no more RIT");
|
||||
ZKAssign.blockUntilNoRIT(zkw);
|
||||
|
||||
return cluster;
|
||||
}
|
||||
private void abortMaster(MiniHBaseCluster cluster) {
|
||||
// Stop the master
|
||||
log("Aborting master");
|
||||
cluster.abortMaster(0);
|
||||
cluster.waitOnMaster(0);
|
||||
log("Master has aborted");
|
||||
}
|
||||
private HRegion getRegionBeingServed(MiniHBaseCluster cluster,
|
||||
HRegionServer regionServer) {
|
||||
Collection<HRegion> onlineRegionsLocalContext = regionServer
|
||||
.getOnlineRegionsLocalContext();
|
||||
Iterator<HRegion> iterator = onlineRegionsLocalContext.iterator();
|
||||
HRegion region = null;
|
||||
while (iterator.hasNext()) {
|
||||
region = iterator.next();
|
||||
if (!region.getRegionInfo().isMetaRegion()
|
||||
&& !region.getRegionInfo().isRootRegion()) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
return region;
|
||||
}
|
||||
private void log(String msg) {
|
||||
LOG.debug("\n\nTRR: " + msg + "\n");
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue