HBASE-5206 Port HBASE-5155 to trunk (Ashutosh Jindal)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1301709 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a388bae569
commit
458a3ff092
|
@ -1587,6 +1587,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
if (isDisabledorDisablingRegionInRIT(region)) {
|
||||
return;
|
||||
}
|
||||
setEnabledTable(region);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2201,6 +2202,14 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
LOG.info("Bulk assigning done");
|
||||
}
|
||||
|
||||
private void setEnabledTable(HRegionInfo hri) {
|
||||
String tableName = hri.getTableNameAsString();
|
||||
boolean isTableEnabled = this.zkTable.isEnabledTable(tableName);
|
||||
if (!isTableEnabled) {
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Assigns all user regions, if any exist. Used during cluster startup.
|
||||
* <p>
|
||||
|
@ -2236,6 +2245,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
} else {
|
||||
// assign regions in round-robin fashion
|
||||
assignUserRegions(new ArrayList<HRegionInfo>(allRegions.keySet()), servers);
|
||||
for (HRegionInfo hri : allRegions.keySet()) {
|
||||
setEnabledTable(hri);
|
||||
}
|
||||
return;
|
||||
}
|
||||
LOG.info("Bulk assigning " + allRegions.size() + " region(s) across " +
|
||||
|
@ -2244,6 +2256,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Use fixed count thread pool assigning.
|
||||
BulkAssigner ba = new StartupBulkAssigner(this.master, bulkPlan, this);
|
||||
ba.bulkAssign();
|
||||
for (HRegionInfo hri : allRegions.keySet()) {
|
||||
setEnabledTable(hri);
|
||||
}
|
||||
LOG.info("Bulk assigning done");
|
||||
}
|
||||
|
||||
|
@ -2437,6 +2452,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
new TreeMap<ServerName, List<Pair<HRegionInfo, Result>>>();
|
||||
// Iterate regions in META
|
||||
for (Result result : results) {
|
||||
boolean disabled = false;
|
||||
boolean disablingOrEnabling = false;
|
||||
Pair<HRegionInfo, ServerName> region = MetaReader.parseCatalogResult(result);
|
||||
if (region == null) continue;
|
||||
HRegionInfo regionInfo = region.getFirst();
|
||||
|
@ -2470,6 +2487,13 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
offlineServers.put(regionLocation, offlineRegions);
|
||||
}
|
||||
offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
|
||||
disabled = checkIfRegionBelongsToDisabled(regionInfo);
|
||||
disablingOrEnabling = addTheTablesInPartialState(this.disablingTables,
|
||||
this.enablingTables, regionInfo, tableName);
|
||||
// need to enable the table if not disabled or disabling or enabling
|
||||
// this will be used in rolling restarts
|
||||
enableTableIfNotDisabledOrDisablingOrEnabling(disabled,
|
||||
disablingOrEnabling, tableName);
|
||||
} else {
|
||||
// Region is being served and on an active server
|
||||
// add only if region not in disabled and enabling table
|
||||
|
@ -2478,21 +2502,37 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
regions.put(regionInfo, regionLocation);
|
||||
addToServers(regionLocation, regionInfo);
|
||||
}
|
||||
addTheTablesInPartialState(this.disablingTables, this.enablingTables, regionInfo,
|
||||
tableName);
|
||||
disablingOrEnabling = addTheTablesInPartialState(this.disablingTables,
|
||||
this.enablingTables, regionInfo, tableName);
|
||||
disabled = checkIfRegionBelongsToDisabled(regionInfo);
|
||||
// need to enable the table if not disabled or disabling or enabling
|
||||
// this will be used in rolling restarts
|
||||
enableTableIfNotDisabledOrDisablingOrEnabling(disabled,
|
||||
disablingOrEnabling, tableName);
|
||||
}
|
||||
}
|
||||
return offlineServers;
|
||||
}
|
||||
|
||||
private void addTheTablesInPartialState(Set<String> disablingTables,
|
||||
private void enableTableIfNotDisabledOrDisablingOrEnabling(boolean disabled,
|
||||
boolean disablingOrEnabling, String tableName) {
|
||||
if (!disabled && !disablingOrEnabling
|
||||
&& !getZKTable().isEnabledTable(tableName)) {
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
private Boolean addTheTablesInPartialState(Set<String> disablingTables,
|
||||
Set<String> enablingTables, HRegionInfo regionInfo,
|
||||
String disablingTableName) {
|
||||
if (checkIfRegionBelongsToDisabling(regionInfo)) {
|
||||
disablingTables.add(disablingTableName);
|
||||
return true;
|
||||
} else if (checkIfRegionsBelongsToEnabling(regionInfo)) {
|
||||
enablingTables.add(disablingTableName);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -3309,4 +3349,16 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
this.threadPoolExecutorService.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
protected void setEnabledTable(String tableName) {
|
||||
try {
|
||||
this.zkTable.setEnabledTable(tableName);
|
||||
} catch (KeeperException e) {
|
||||
// here we can abort as it is the start up flow
|
||||
String errorMsg = "Unable to ensure that the table " + tableName
|
||||
+ " will be" + " enabled because of a ZooKeeper issue";
|
||||
LOG.error(errorMsg);
|
||||
this.master.abort(errorMsg, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -626,6 +626,9 @@ Server {
|
|||
this.assignmentManager.regionOnline(HRegionInfo.ROOT_REGIONINFO,
|
||||
this.catalogTracker.getRootLocation());
|
||||
}
|
||||
// Enable the ROOT table if on process fail over the RS containing ROOT
|
||||
// was active.
|
||||
enableCatalogTables(Bytes.toString(HConstants.ROOT_TABLE_NAME));
|
||||
LOG.info("-ROOT- assigned=" + assigned + ", rit=" + rit +
|
||||
", location=" + catalogTracker.getRootLocation());
|
||||
|
||||
|
@ -651,12 +654,19 @@ Server {
|
|||
this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
|
||||
this.catalogTracker.getMetaLocation());
|
||||
}
|
||||
enableCatalogTables(Bytes.toString(HConstants.META_TABLE_NAME));
|
||||
LOG.info(".META. assigned=" + assigned + ", rit=" + rit +
|
||||
", location=" + catalogTracker.getMetaLocation());
|
||||
status.setStatus("META and ROOT assigned.");
|
||||
return assigned;
|
||||
}
|
||||
|
||||
private void enableCatalogTables(String catalogTableName) {
|
||||
if (!this.assignmentManager.getZKTable().isEnabledTable(catalogTableName)) {
|
||||
this.assignmentManager.setEnabledTable(catalogTableName);
|
||||
}
|
||||
}
|
||||
|
||||
void fixupDaughters(final MonitoredTask status) throws IOException {
|
||||
final Map<HRegionInfo, Result> offlineSplitParents =
|
||||
new HashMap<HRegionInfo, Result>();
|
||||
|
|
|
@ -81,8 +81,8 @@ public class DeleteTableHandler extends TableEventHandler {
|
|||
this.masterServices.getTableDescriptors().remove(Bytes.toString(tableName));
|
||||
|
||||
// If entry for this table in zk, and up in AssignmentManager, remove it.
|
||||
// Call to undisableTable does this. TODO: Make a more formal purge table.
|
||||
am.getZKTable().setEnabledTable(Bytes.toString(tableName));
|
||||
|
||||
am.getZKTable().setDeletedTable(Bytes.toString(tableName));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -327,13 +327,27 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
public static boolean processDeadRegion(HRegionInfo hri, Result result,
|
||||
AssignmentManager assignmentManager, CatalogTracker catalogTracker)
|
||||
throws IOException {
|
||||
boolean tablePresent = assignmentManager.getZKTable().isTablePresent(
|
||||
hri.getTableNameAsString());
|
||||
if (!tablePresent) {
|
||||
LOG.info("The table " + hri.getTableNameAsString()
|
||||
+ " was deleted. Hence not proceeding.");
|
||||
return false;
|
||||
}
|
||||
// If table is not disabled but the region is offlined,
|
||||
boolean disabled = assignmentManager.getZKTable().isDisabledTable(
|
||||
hri.getTableNameAsString());
|
||||
if (disabled) return false;
|
||||
if (disabled){
|
||||
LOG.info("The table " + hri.getTableNameAsString()
|
||||
+ " was disabled. Hence not proceeding.");
|
||||
return false;
|
||||
}
|
||||
if (hri.isOffline() && hri.isSplit()) {
|
||||
LOG.debug("Offlined and split region " + hri.getRegionNameAsString() +
|
||||
"; checking daughter presence");
|
||||
if (MetaReader.getRegion(catalogTracker, hri.getRegionName()) == null) {
|
||||
return false;
|
||||
}
|
||||
fixupDaughters(result, assignmentManager, catalogTracker);
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -218,7 +218,7 @@ public class ZKTable {
|
|||
public boolean checkEnabledAndSetDisablingTable(final String tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (!isEnabledTable(tableName)) {
|
||||
if (this.cache.get(tableName) != null && !isEnabledTable(tableName)) {
|
||||
return false;
|
||||
}
|
||||
setTableState(tableName, TableState.DISABLING);
|
||||
|
@ -267,10 +267,7 @@ public class ZKTable {
|
|||
}
|
||||
|
||||
public boolean isEnabledTable(String tableName) {
|
||||
synchronized (this.cache) {
|
||||
// No entry in cache means enabled table.
|
||||
return !this.cache.containsKey(tableName);
|
||||
}
|
||||
return isTableState(tableName, TableState.ENABLED);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -285,7 +282,7 @@ public class ZKTable {
|
|||
public static boolean isEnabledTable(final ZooKeeperWatcher zkw,
|
||||
final String tableName)
|
||||
throws KeeperException {
|
||||
return getTableState(zkw, tableName) == null;
|
||||
return getTableState(zkw, tableName) == TableState.ENABLED;
|
||||
}
|
||||
|
||||
public boolean isDisablingOrDisabledTable(final String tableName) {
|
||||
|
@ -337,23 +334,47 @@ public class ZKTable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Enables the table in zookeeper. Fails silently if the
|
||||
* Deletes the table in zookeeper. Fails silently if the
|
||||
* table is not currently disabled in zookeeper. Sets no watches.
|
||||
* @param tableName
|
||||
* @throws KeeperException unexpected zookeeper exception
|
||||
*/
|
||||
public void setEnabledTable(final String tableName)
|
||||
public void setDeletedTable(final String tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (this.cache.remove(tableName) == null) {
|
||||
LOG.warn("Moving table " + tableName + " state to enabled but was " +
|
||||
"already enabled");
|
||||
LOG.warn("Moving table " + tableName + " state to deleted but was " +
|
||||
"already deleted");
|
||||
}
|
||||
ZKUtil.deleteNodeFailSilent(this.watcher,
|
||||
ZKUtil.joinZNode(this.watcher.tableZNode, tableName));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the ENABLED state in the cache and deletes the zookeeper node. Fails
|
||||
* silently if the node is not in enabled in zookeeper
|
||||
*
|
||||
* @param tableName
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public void setEnabledTable(final String tableName) throws KeeperException {
|
||||
setTableState(tableName, TableState.ENABLED);
|
||||
}
|
||||
|
||||
/**
|
||||
* check if table is present .
|
||||
*
|
||||
* @param tableName
|
||||
* @return true if the table is present
|
||||
*/
|
||||
public boolean isTablePresent(final String tableName) {
|
||||
synchronized (this.cache) {
|
||||
TableState state = this.cache.get(tableName);
|
||||
return !(state == null);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as disabled in zookeeper.
|
||||
* @return Set of disabled tables, empty Set if none
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
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;
|
||||
|
@ -61,6 +62,7 @@ public class TestDrainingServer {
|
|||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.startMiniCluster(5);
|
||||
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
|
||||
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLENAME);
|
||||
htd.addFamily(new HColumnDescriptor(FAMILY));
|
||||
TEST_UTIL.createMultiRegionsInMeta(TEST_UTIL.getConfiguration(), htd,
|
||||
|
@ -73,6 +75,7 @@ public class TestDrainingServer {
|
|||
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
|
||||
admin.disableTable(TABLENAME);
|
||||
admin.enableTable(TABLENAME);
|
||||
ZKAssign.blockUntilNoRIT(zkw);
|
||||
// Assert that every regionserver has some regions on it.
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
|
||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||
|
|
|
@ -49,6 +49,8 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtilsForTests;
|
||||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.*;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
|
@ -222,6 +224,9 @@ public class TestAdmin {
|
|||
ht.get(get);
|
||||
|
||||
this.admin.disableTable(table);
|
||||
assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
|
||||
.getMaster().getAssignmentManager().getZKTable().isDisabledTable(
|
||||
Bytes.toString(table)));
|
||||
|
||||
// Test that table is disabled
|
||||
get = new Get(row);
|
||||
|
@ -236,6 +241,9 @@ public class TestAdmin {
|
|||
}
|
||||
assertTrue(ok);
|
||||
this.admin.enableTable(table);
|
||||
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
|
||||
.getMaster().getAssignmentManager().getZKTable().isEnabledTable(
|
||||
Bytes.toString(table)));
|
||||
|
||||
// Test that table is enabled
|
||||
try {
|
||||
|
@ -307,6 +315,9 @@ public class TestAdmin {
|
|||
HConstants.CATALOG_FAMILY).close();
|
||||
tables = this.admin.listTables();
|
||||
assertEquals(numTables + 1, tables.length);
|
||||
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
|
||||
.getMaster().getAssignmentManager().getZKTable().isEnabledTable(
|
||||
"testCreateTable"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -953,10 +964,14 @@ public class TestAdmin {
|
|||
new HColumnDescriptor("/cfamily/name");
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout=36000)
|
||||
public void testEnableDisableAddColumnDeleteColumn() throws Exception {
|
||||
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
|
||||
byte [] tableName = Bytes.toBytes("testMasterAdmin");
|
||||
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
|
||||
while (!ZKTable.isEnabledTable(zkw, "testMasterAdmin")) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
this.admin.disableTable(tableName);
|
||||
try {
|
||||
new HTable(TEST_UTIL.getConfiguration(), tableName);
|
||||
|
|
|
@ -69,6 +69,8 @@ public class TestMaster {
|
|||
HMaster m = cluster.getMaster();
|
||||
|
||||
HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME);
|
||||
assertTrue(m.assignmentManager.getZKTable().isEnabledTable
|
||||
(Bytes.toString(TABLENAME)));
|
||||
TEST_UTIL.loadTable(ht, FAMILYNAME);
|
||||
ht.close();
|
||||
|
||||
|
|
|
@ -743,6 +743,8 @@ public class TestMasterFailover {
|
|||
master.assignRegion(hri);
|
||||
}
|
||||
|
||||
assertTrue(" Table must be enabled.", master.getAssignmentManager()
|
||||
.getZKTable().isEnabledTable("enabledTable"));
|
||||
// we also need regions assigned out on the dead server
|
||||
List<HRegionInfo> enabledAndOnDeadRegions = new ArrayList<HRegionInfo>();
|
||||
enabledAndOnDeadRegions.add(enabledRegions.remove(0));
|
||||
|
@ -788,6 +790,9 @@ public class TestMasterFailover {
|
|||
ZKTable zktable = new ZKTable(zkw);
|
||||
zktable.setDisabledTable(Bytes.toString(disabledTable));
|
||||
|
||||
assertTrue(" The enabled table should be identified on master fail over.",
|
||||
zktable.isEnabledTable("enabledTable"));
|
||||
|
||||
/*
|
||||
* ZK = CLOSING
|
||||
*/
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -96,6 +97,9 @@ public class TestMasterRestartAfterDisablingTable {
|
|||
cluster.hbaseCluster.waitOnMaster(activeMaster);
|
||||
cluster.waitForActiveAndReadyMaster();
|
||||
|
||||
assertTrue("The table should not be in enabled state", cluster.getMaster()
|
||||
.getAssignmentManager().getZKTable().isDisablingOrDisabledTable(
|
||||
"tableRestart"));
|
||||
log("Enabling table\n");
|
||||
// Need a new Admin, the previous one is on the old master
|
||||
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
|
||||
|
@ -108,6 +112,8 @@ public class TestMasterRestartAfterDisablingTable {
|
|||
assertEquals(
|
||||
"The assigned regions were not onlined after master switch except for the catalog tables.",
|
||||
6, regions.size());
|
||||
assertTrue("The table should be in enabled state", cluster.getMaster()
|
||||
.getAssignmentManager().getZKTable().isEnabledTable("tableRestart"));
|
||||
ht.close();
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
|
|
@ -68,29 +68,42 @@ public class TestZKTable {
|
|||
ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
|
||||
name, abortable, true);
|
||||
ZKTable zkt = new ZKTable(zkw);
|
||||
assertTrue(zkt.isEnabledTable(name));
|
||||
assertFalse(zkt.isEnabledTable(name));
|
||||
assertFalse(zkt.isDisablingTable(name));
|
||||
assertFalse(zkt.isDisabledTable(name));
|
||||
assertFalse(zkt.isEnablingTable(name));
|
||||
assertFalse(zkt.isDisablingOrDisabledTable(name));
|
||||
assertFalse(zkt.isDisabledOrEnablingTable(name));
|
||||
assertFalse(zkt.isTablePresent(name));
|
||||
zkt.setDisablingTable(name);
|
||||
assertTrue(zkt.isDisablingTable(name));
|
||||
assertTrue(zkt.isDisablingOrDisabledTable(name));
|
||||
assertFalse(zkt.getDisabledTables().contains(name));
|
||||
assertTrue(zkt.isTablePresent(name));
|
||||
zkt.setDisabledTable(name);
|
||||
assertTrue(zkt.isDisabledTable(name));
|
||||
assertTrue(zkt.isDisablingOrDisabledTable(name));
|
||||
assertFalse(zkt.isDisablingTable(name));
|
||||
assertTrue(zkt.getDisabledTables().contains(name));
|
||||
assertTrue(zkt.isTablePresent(name));
|
||||
zkt.setEnablingTable(name);
|
||||
assertTrue(zkt.isEnablingTable(name));
|
||||
assertTrue(zkt.isDisabledOrEnablingTable(name));
|
||||
assertFalse(zkt.isDisabledTable(name));
|
||||
assertFalse(zkt.getDisabledTables().contains(name));
|
||||
assertTrue(zkt.isTablePresent(name));
|
||||
zkt.setEnabledTable(name);
|
||||
assertTrue(zkt.isEnabledTable(name));
|
||||
assertFalse(zkt.isEnablingTable(name));
|
||||
assertTrue(zkt.isTablePresent(name));
|
||||
zkt.setDeletedTable(name);
|
||||
assertFalse(zkt.isEnabledTable(name));
|
||||
assertFalse(zkt.isDisablingTable(name));
|
||||
assertFalse(zkt.isDisabledTable(name));
|
||||
assertFalse(zkt.isEnablingTable(name));
|
||||
assertFalse(zkt.isDisablingOrDisabledTable(name));
|
||||
assertFalse(zkt.isDisabledOrEnablingTable(name));
|
||||
assertFalse(zkt.isTablePresent(name));
|
||||
}
|
||||
|
||||
@org.junit.Rule
|
||||
|
|
Loading…
Reference in New Issue