HBASE-4729 Clash between region unassign and splitting kills the master
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1211197 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ff8e1a53c5
commit
cb4320232d
|
@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
|||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
|
||||
import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
|
||||
|
@ -84,7 +83,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
|||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.zookeeper.AsyncCallback;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
import org.apache.zookeeper.KeeperException.NodeExistsException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
/**
|
||||
|
@ -246,6 +245,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
/**
|
||||
* Add a regionPlan for the specified region.
|
||||
* @param encodedName
|
||||
* @param plan
|
||||
*/
|
||||
public void addPlan(String encodedName, RegionPlan plan) {
|
||||
synchronized (regionPlans) {
|
||||
|
@ -404,7 +405,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
boolean intransistion =
|
||||
processRegionInTransition(hri.getEncodedName(), hri, null);
|
||||
if (!intransistion) return intransistion;
|
||||
debugLog(hri, "Waiting on " + HRegionInfo.prettyPrint(hri.getEncodedName()));
|
||||
LOG.debug("Waiting on " + HRegionInfo.prettyPrint(hri.getEncodedName()));
|
||||
synchronized(this.regionsInTransition) {
|
||||
while (!this.master.isStopped() &&
|
||||
this.regionsInTransition.containsKey(hri.getEncodedName())) {
|
||||
|
@ -541,7 +542,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
throws KeeperException {
|
||||
// If was on dead server, its closed now. Force to OFFLINE and then
|
||||
// handle it like a close; this will get it reassigned if appropriate
|
||||
debugLog(hri, "RIT " + hri.getEncodedName() + " in state=" +
|
||||
LOG.debug("RIT " + hri.getEncodedName() + " in state=" +
|
||||
oldData.getEventType() + " was on deadserver; forcing offline");
|
||||
ZKAssign.createOrForceNodeOffline(this.watcher, hri,
|
||||
this.master.getServerName());
|
||||
|
@ -957,12 +958,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void nodeDeleted(String path) {
|
||||
// Added so we notice when ephemeral nodes go away; in particular,
|
||||
// SPLITTING or SPLIT nodes added by a regionserver splitting.
|
||||
public void nodeDeleted(final String path) {
|
||||
if (path.startsWith(this.watcher.assignmentZNode)) {
|
||||
String regionName =
|
||||
ZKAssign.getRegionName(this.master.getZooKeeper(), path);
|
||||
String regionName = ZKAssign.getRegionName(this.master.getZooKeeper(), path);
|
||||
RegionState rs = this.regionsInTransition.get(regionName);
|
||||
if (rs != null) {
|
||||
HRegionInfo regionInfo = rs.getRegion();
|
||||
|
@ -980,7 +978,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void makeRegionOnline(RegionState rs, HRegionInfo regionInfo) {
|
||||
regionOnline(regionInfo, rs.serverName);
|
||||
LOG.info("The master has opened the region "
|
||||
|
@ -988,14 +986,13 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
+ rs.serverName);
|
||||
if (this.getZKTable().isDisablingOrDisabledTable(
|
||||
regionInfo.getTableNameAsString())) {
|
||||
debugLog(regionInfo, "Opened region "
|
||||
LOG.debug("Opened region "
|
||||
+ regionInfo.getRegionNameAsString() + " but "
|
||||
+ "this table is disabled, triggering close of region");
|
||||
unassign(regionInfo);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* New unassigned node has been created.
|
||||
*
|
||||
|
@ -1029,7 +1026,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @param regionInfo
|
||||
* @param sn
|
||||
*/
|
||||
public void regionOnline(HRegionInfo regionInfo, ServerName sn) {
|
||||
void regionOnline(HRegionInfo regionInfo, ServerName sn) {
|
||||
synchronized (this.regionsInTransition) {
|
||||
RegionState rs =
|
||||
this.regionsInTransition.remove(regionInfo.getEncodedName());
|
||||
|
@ -1185,7 +1182,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
*/
|
||||
public void assign(HRegionInfo region, boolean setOfflineInZK,
|
||||
boolean forceNewPlan, boolean hijack) {
|
||||
//If hijack is true do not call disableRegionIfInRIT as
|
||||
// If hijack is true do not call disableRegionIfInRIT as
|
||||
// we have not yet moved the znode to OFFLINE state.
|
||||
if (!hijack && isDisabledorDisablingRegionInRIT(region)) {
|
||||
return;
|
||||
|
@ -1440,13 +1437,11 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
RegionPlan plan = getRegionPlan(state, forceNewPlan);
|
||||
if (plan == null) {
|
||||
debugLog(state.getRegion(),
|
||||
"Unable to determine a plan to assign " + state);
|
||||
LOG.debug("Unable to determine a plan to assign " + state);
|
||||
return; // Should get reassigned later when RIT times out.
|
||||
}
|
||||
try {
|
||||
debugLog(state.getRegion(),
|
||||
"Assigning region " + state.getRegion().getRegionNameAsString() +
|
||||
LOG.debug("Assigning region " + state.getRegion().getRegionNameAsString() +
|
||||
" to " + plan.getDestination().toString());
|
||||
// Transition RegionState to PENDING_OPEN
|
||||
state.update(RegionState.State.PENDING_OPEN, System.currentTimeMillis(),
|
||||
|
@ -1459,8 +1454,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Remove region from in-memory transition and unassigned node from ZK
|
||||
// While trying to enable the table the regions of the table were
|
||||
// already enabled.
|
||||
debugLog(state.getRegion(),
|
||||
"ALREADY_OPENED region " + state.getRegion().getRegionNameAsString() +
|
||||
LOG.debug("ALREADY_OPENED region " + state.getRegion().getRegionNameAsString() +
|
||||
" to " + plan.getDestination().toString());
|
||||
String encodedRegionName = state.getRegion()
|
||||
.getEncodedName();
|
||||
|
@ -1512,14 +1506,6 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
}
|
||||
|
||||
private void debugLog(HRegionInfo region, String string) {
|
||||
if (region.isMetaTable()) {
|
||||
LOG.info(string);
|
||||
} else {
|
||||
LOG.debug(string);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
|
||||
String tableName = region.getTableNameAsString();
|
||||
boolean disabled = this.zkTable.isDisabledTable(tableName);
|
||||
|
@ -1674,7 +1660,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
if (newPlan) {
|
||||
debugLog(state.getRegion(), "No previous transition plan was found (or we are ignoring " +
|
||||
LOG.debug("No previous transition plan was found (or we are ignoring " +
|
||||
"an existing plan) for " + state.getRegion().getRegionNameAsString() +
|
||||
" so generated a random one; " + randomPlan + "; " +
|
||||
serverManager.countOfRegionServers() +
|
||||
|
@ -1682,7 +1668,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
", exclude=" + drainingServers.size() + ") available servers");
|
||||
return randomPlan;
|
||||
}
|
||||
debugLog(state.getRegion(), "Using pre-existing plan for region " +
|
||||
LOG.debug("Using pre-existing plan for region " +
|
||||
state.getRegion().getRegionNameAsString() + "; plan=" + existingPlan);
|
||||
return existingPlan;
|
||||
}
|
||||
|
@ -1721,7 +1707,11 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
/**
|
||||
* Unassigns the specified region.
|
||||
* <p>
|
||||
* Updates the RegionState and sends the CLOSE RPC.
|
||||
* Updates the RegionState and sends the CLOSE RPC unless region is being
|
||||
* split by regionserver; then the unassign fails (silently) because we
|
||||
* presume the region being unassigned no longer exists (its been split out
|
||||
* of existence). TODO: What to do if split fails and is rolled back and
|
||||
* parent is revivified?
|
||||
* <p>
|
||||
* If a RegionPlan is already set, it will remain.
|
||||
*
|
||||
|
@ -1734,7 +1724,11 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
/**
|
||||
* Unassigns the specified region.
|
||||
* <p>
|
||||
* Updates the RegionState and sends the CLOSE RPC.
|
||||
* Updates the RegionState and sends the CLOSE RPC unless region is being
|
||||
* split by regionserver; then the unassign fails (silently) because we
|
||||
* presume the region being unassigned no longer exists (its been split out
|
||||
* of existence). TODO: What to do if split fails and is rolled back and
|
||||
* parent is revivified?
|
||||
* <p>
|
||||
* If a RegionPlan is already set, it will remain.
|
||||
*
|
||||
|
@ -1742,12 +1736,13 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @param force if region should be closed even if already closing
|
||||
*/
|
||||
public void unassign(HRegionInfo region, boolean force) {
|
||||
debugLog(region, "Starting unassignment of region " +
|
||||
// TODO: Method needs refactoring. Ugly buried returns throughout. Beware!
|
||||
LOG.debug("Starting unassignment of region " +
|
||||
region.getRegionNameAsString() + " (offlining)");
|
||||
synchronized (this.regions) {
|
||||
// Check if this region is currently assigned
|
||||
if (!regions.containsKey(region)) {
|
||||
debugLog(region, "Attempted to unassign region " +
|
||||
LOG.debug("Attempted to unassign region " +
|
||||
region.getRegionNameAsString() + " but it is not " +
|
||||
"currently assigned anywhere");
|
||||
return;
|
||||
|
@ -1759,27 +1754,49 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
synchronized (regionsInTransition) {
|
||||
state = regionsInTransition.get(encodedName);
|
||||
if (state == null) {
|
||||
|
||||
// Create the znode in CLOSING state
|
||||
try {
|
||||
ZKAssign.createNodeClosing(
|
||||
master.getZooKeeper(), region, master.getServerName());
|
||||
} catch (KeeperException e) {
|
||||
if (e instanceof NodeExistsException) {
|
||||
// Handle race between master initiated close and regionserver
|
||||
// orchestrated splitting. See if existing node is in a
|
||||
// SPLITTING or SPLIT state. If so, the regionserver started
|
||||
// an op on node before we could get our CLOSING in. Deal.
|
||||
NodeExistsException nee = (NodeExistsException)e;
|
||||
String path = nee.getPath();
|
||||
try {
|
||||
if (isSplitOrSplitting(path)) {
|
||||
LOG.debug(path + " is SPLIT or SPLITTING; " +
|
||||
"skipping unassign because region no longer exists -- its split");
|
||||
return;
|
||||
}
|
||||
} catch (KeeperException.NoNodeException ke) {
|
||||
LOG.warn("Failed getData on SPLITTING/SPLIT at " + path +
|
||||
"; presuming split and that the region to unassign, " +
|
||||
encodedName + ", no longer exists -- confirm", ke);
|
||||
return;
|
||||
} catch (KeeperException ke) {
|
||||
LOG.error("Unexpected zk state", ke);
|
||||
ke = e;
|
||||
}
|
||||
}
|
||||
// If we get here, don't understand whats going on -- abort.
|
||||
master.abort("Unexpected ZK exception creating node CLOSING", e);
|
||||
return;
|
||||
}
|
||||
state = new RegionState(region, RegionState.State.PENDING_CLOSE);
|
||||
regionsInTransition.put(encodedName, state);
|
||||
} else if (force && (state.isPendingClose() || state.isClosing())) {
|
||||
debugLog(region,
|
||||
"Attempting to unassign region " + region.getRegionNameAsString() +
|
||||
" which is already " + state.getState() +
|
||||
" but forcing to send a CLOSE RPC again ");
|
||||
LOG.debug("Attempting to unassign region " + region.getRegionNameAsString() +
|
||||
" which is already " + state.getState() +
|
||||
" but forcing to send a CLOSE RPC again ");
|
||||
state.update(state.getState());
|
||||
} else {
|
||||
debugLog(region, "Attempting to unassign region " +
|
||||
LOG.debug("Attempting to unassign region " +
|
||||
region.getRegionNameAsString() + " but it is " +
|
||||
"already in transition (" + state.getState() + ")");
|
||||
"already in transition (" + state.getState() + ", force=" + force + ")");
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
@ -1792,7 +1809,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// TODO: We should consider making this look more like it does for the
|
||||
// region open where we catch all throwables and never abort
|
||||
if (serverManager.sendRegionClose(server, state.getRegion())) {
|
||||
debugLog(region, "Sent CLOSE to " + server + " for region " +
|
||||
LOG.debug("Sent CLOSE to " + server + " for region " +
|
||||
region.getRegionNameAsString());
|
||||
return;
|
||||
}
|
||||
|
@ -1826,7 +1843,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
// RS is already processing this region, only need to update the timestamp
|
||||
if (t instanceof RegionAlreadyInTransitionException) {
|
||||
debugLog(region, "update " + state + " the timestamp.");
|
||||
LOG.debug("update " + state + " the timestamp.");
|
||||
state.update(state.getState());
|
||||
}
|
||||
}
|
||||
|
@ -1836,6 +1853,28 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param path
|
||||
* @return True if znode is in SPLIT or SPLITTING state.
|
||||
* @throws KeeperException Can happen if the znode went away in meantime.
|
||||
*/
|
||||
private boolean isSplitOrSplitting(final String path) throws KeeperException {
|
||||
boolean result = false;
|
||||
// This may fail if the SPLIT or SPLITTING znode gets cleaned up before we
|
||||
// can get data from it.
|
||||
RegionTransitionData data = ZKAssign.getData(master.getZooKeeper(), path);
|
||||
EventType evt = data.getEventType();
|
||||
switch (evt) {
|
||||
case RS_ZK_REGION_SPLIT:
|
||||
case RS_ZK_REGION_SPLITTING:
|
||||
result = true;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Waits until the specified region has completed assignment.
|
||||
* <p>
|
||||
|
@ -2938,6 +2977,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
/**
|
||||
* Check whether the RegionServer is online.
|
||||
* @param serverName
|
||||
* @return True if online.
|
||||
*/
|
||||
public boolean isServerOnline(ServerName serverName) {
|
||||
return this.serverManager.isServerOnline(serverName);
|
||||
|
|
|
@ -818,6 +818,7 @@ public class SplitTransaction {
|
|||
* @param region region to be created as offline
|
||||
* @param serverName server event originates from
|
||||
* @return Version of znode created.
|
||||
* @throws KeeperException
|
||||
* @throws IOException
|
||||
*/
|
||||
private static int createNodeSplitting(final ZooKeeperWatcher zkw,
|
||||
|
|
|
@ -81,8 +81,6 @@ public class ZKTable {
|
|||
|
||||
/**
|
||||
* Gets a list of all the tables set as disabled in zookeeper.
|
||||
* @param zkw
|
||||
* @return list of disabled tables, empty list if none
|
||||
* @throws KeeperException
|
||||
*/
|
||||
private void populateTableStates()
|
||||
|
@ -90,6 +88,7 @@ public class ZKTable {
|
|||
synchronized (this.cache) {
|
||||
List<String> children =
|
||||
ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode);
|
||||
if (children == null) return;
|
||||
for (String child: children) {
|
||||
TableState state = getTableState(this.watcher, child);
|
||||
if (state != null) this.cache.put(child, state);
|
||||
|
|
|
@ -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.hbase.master;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
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.KeeperException.NodeExistsException;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
|
||||
/**
|
||||
* Test {@link AssignmentManager}
|
||||
*/
|
||||
@Category(SmallTests.class)
|
||||
public class TestAssignmentManager {
|
||||
private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
|
||||
private static final ServerName RANDOM_SERVERNAME =
|
||||
new ServerName("example.org", 1234, 5678);
|
||||
private Server server;
|
||||
private ServerManager serverManager;
|
||||
private CatalogTracker ct;
|
||||
private ExecutorService executor;
|
||||
private ZooKeeperWatcher watcher;
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
HTU.startMiniZKCluster();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void afterClass() throws IOException {
|
||||
HTU.shutdownMiniZKCluster();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() throws ZooKeeperConnectionException, IOException {
|
||||
// Mock a Server. Have it return a legit Configuration and ZooKeeperWatcher.
|
||||
// If abort is called, be sure to fail the test (don't just swallow it
|
||||
// silently as is mockito default).
|
||||
this.server = Mockito.mock(Server.class);
|
||||
Mockito.when(server.getConfiguration()).thenReturn(HTU.getConfiguration());
|
||||
this.watcher =
|
||||
new ZooKeeperWatcher(HTU.getConfiguration(), "mocked server", this.server, true);
|
||||
Mockito.when(server.getZooKeeper()).thenReturn(this.watcher);
|
||||
Mockito.doThrow(new RuntimeException("Aborted")).
|
||||
when(server).abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
|
||||
// Mock a ServerManager. Say the server RANDOME_SERVERNAME is online.
|
||||
// Also, when someone sends sendRegionClose, say true it succeeded.
|
||||
this.serverManager = Mockito.mock(ServerManager.class);
|
||||
Mockito.when(this.serverManager.isServerOnline(RANDOM_SERVERNAME)).thenReturn(true);
|
||||
this.ct = Mockito.mock(CatalogTracker.class);
|
||||
this.executor = Mockito.mock(ExecutorService.class);
|
||||
}
|
||||
|
||||
@After
|
||||
public void after() {
|
||||
if (this.watcher != null) this.watcher.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnassignWithSplitAtSameTime() throws KeeperException, IOException {
|
||||
// Region to use in test.
|
||||
final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
|
||||
// First amend the servermanager mock so that when we do send close of the
|
||||
// first meta region on RANDOM_SERVERNAME, it will return true rather than
|
||||
// default null.
|
||||
Mockito.when(this.serverManager.sendRegionClose(RANDOM_SERVERNAME, hri)).thenReturn(true);
|
||||
// Create an AM.
|
||||
AssignmentManager am =
|
||||
new AssignmentManager(this.server, this.serverManager, this.ct, this.executor);
|
||||
try {
|
||||
// First make sure my mock up basically works. Unassign a region.
|
||||
unassign(am, RANDOM_SERVERNAME, hri);
|
||||
// This delete will fail if the previous unassign did wrong thing.
|
||||
ZKAssign.deleteClosingNode(this.watcher, hri);
|
||||
// Now put a SPLITTING region in the way. I don't have to assert it
|
||||
// go put in place. This method puts it in place then asserts it still
|
||||
// owns it by moving state from SPLITTING to SPLITTING.
|
||||
int version = createNodeSplitting(this.watcher, hri, RANDOM_SERVERNAME);
|
||||
// Now, retry the unassign with the SPLTTING in place. It should just
|
||||
// complete without fail; a sort of 'silent' recognition that the
|
||||
// region to unassign has been split and no longer exists: TOOD: what if
|
||||
// the split fails and the parent region comes back to life?
|
||||
unassign(am, RANDOM_SERVERNAME, hri);
|
||||
// This transition should fail if the znode has been messed with.
|
||||
ZKAssign.transitionNode(this.watcher, hri, RANDOM_SERVERNAME,
|
||||
EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);
|
||||
assertTrue(am.isRegionInTransition(hri) == null);
|
||||
} finally {
|
||||
am.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new ephemeral node in the SPLITTING state for the specified region.
|
||||
* Create it ephemeral in case regionserver dies mid-split.
|
||||
*
|
||||
* <p>Does not transition nodes from other states. If a node already exists
|
||||
* for this region, a {@link NodeExistsException} will be thrown.
|
||||
*
|
||||
* @param zkw zk reference
|
||||
* @param region region to be created as offline
|
||||
* @param serverName server event originates from
|
||||
* @return Version of znode created.
|
||||
* @throws KeeperException
|
||||
* @throws IOException
|
||||
*/
|
||||
// Copied from SplitTransaction rather than open the method over there in
|
||||
// the regionserver package.
|
||||
private static int createNodeSplitting(final ZooKeeperWatcher zkw,
|
||||
final HRegionInfo region, final ServerName serverName)
|
||||
throws KeeperException, IOException {
|
||||
RegionTransitionData data =
|
||||
new RegionTransitionData(EventType.RS_ZK_REGION_SPLITTING,
|
||||
region.getRegionName(), serverName);
|
||||
|
||||
String node = ZKAssign.getNodeName(zkw, region.getEncodedName());
|
||||
if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, data.getBytes())) {
|
||||
throw new IOException("Failed create of ephemeral " + node);
|
||||
}
|
||||
// Transition node from SPLITTING to SPLITTING and pick up version so we
|
||||
// can be sure this znode is ours; version is needed deleting.
|
||||
return transitionNodeSplitting(zkw, region, serverName, -1);
|
||||
}
|
||||
|
||||
// Copied from SplitTransaction rather than open the method over there in
|
||||
// the regionserver package.
|
||||
private static int transitionNodeSplitting(final ZooKeeperWatcher zkw,
|
||||
final HRegionInfo parent,
|
||||
final ServerName serverName, final int version)
|
||||
throws KeeperException, IOException {
|
||||
return ZKAssign.transitionNode(zkw, parent, serverName,
|
||||
EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);
|
||||
}
|
||||
|
||||
private void unassign(final AssignmentManager am, final ServerName sn,
|
||||
final HRegionInfo hri) {
|
||||
// Before I can unassign a region, I need to set it online.
|
||||
am.regionOnline(hri, sn);
|
||||
// Unassign region.
|
||||
am.unassign(hri);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue