HBASE-9547 Master could fail start if region server with system table is down

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1523522 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
jxiang 2013-09-16 01:41:12 +00:00
parent 1f969f6155
commit 704a00b544
5 changed files with 164 additions and 233 deletions

View File

@ -456,21 +456,24 @@ public class AssignmentManager extends ZooKeeperListener {
.getRequeuedDeadServers().isEmpty());
if (!failover) {
// Run through all regions. If they are not assigned and not in RIT, then
// its a clean cluster startup, else its a failover.
// If any one region except meta is assigned, it's a failover.
Map<HRegionInfo, ServerName> regions = regionStates.getRegionAssignments();
for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
if (!e.getKey().getTable().isSystemTable()
&& e.getValue() != null) {
LOG.debug("Found " + e + " out on cluster");
for (HRegionInfo hri: regions.keySet()) {
if (!hri.isMetaTable()) {
LOG.debug("Found " + hri + " out on cluster");
failover = true;
break;
}
if (nodes.contains(e.getKey().getEncodedName())) {
LOG.debug("Found " + e.getKey().getRegionNameAsString() + " in RITs");
// Could be a meta region.
failover = true;
break;
}
if (!failover) {
// If any one region except meta is in transition, it's a failover.
for (String encodedName: nodes) {
RegionState state = regionStates.getRegionState(encodedName);
if (state != null && !state.getRegion().isMetaRegion()) {
LOG.debug("Found " + state.getRegion().getRegionNameAsString() + " in RITs");
failover = true;
break;
}
}
}
}
@ -2585,17 +2588,7 @@ public class AssignmentManager extends ZooKeeperListener {
new SnapshotOfRegionAssignmentFromMeta(catalogTracker, disabledOrDisablingOrEnabling, true);
snapshotOfRegionAssignment.initialize();
allRegions = snapshotOfRegionAssignment.getRegionToRegionServerMap();
if (allRegions == null) return;
//remove system tables because they would have been assigned earlier
for(Iterator<HRegionInfo> iter = allRegions.keySet().iterator();
iter.hasNext();) {
if (iter.next().getTable().isSystemTable()) {
iter.remove();
}
}
if (allRegions.isEmpty()) return;
if (allRegions == null || allRegions.isEmpty()) return;
// Determine what type of assignment to do on startup
boolean retainAssignment = server.getConfiguration().
@ -2670,8 +2663,6 @@ public class AssignmentManager extends ZooKeeperListener {
HRegionInfo regionInfo = region.getFirst();
ServerName regionLocation = region.getSecond();
if (regionInfo == null) continue;
TableName tableName = regionInfo.getTable();
if (tableName.isSystemTable()) continue;
regionStates.createRegionState(regionInfo);
if (regionStates.isRegionInState(regionInfo, State.SPLIT)) {
// Split is considered to be completed. If the split znode still
@ -2680,6 +2671,7 @@ public class AssignmentManager extends ZooKeeperListener {
+ " split is completed. Hence need not add to regions list");
continue;
}
TableName tableName = regionInfo.getTable();
if (regionLocation == null) {
// regionLocation could be null if createTable didn't finish properly.
// When createTable is in progress, HMaster restarts.

View File

@ -29,7 +29,6 @@ import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -43,7 +42,6 @@ import java.util.concurrent.atomic.AtomicReference;
import javax.management.ObjectName;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -85,13 +83,11 @@ import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorType;
import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
import org.apache.hadoop.hbase.ipc.RpcScheduler;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.RequestContext;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.ipc.SimpleRpcScheduler;
import org.apache.hadoop.hbase.master.balancer.BalancerChore;
import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
@ -215,7 +211,6 @@ import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKTable;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -875,12 +870,6 @@ MasterServices, Server {
this.fileSystemManager.splitMetaLog(previouslyFailedMetaRSs);
}
status.setStatus("Assigning System tables");
// Make sure system tables are assigned before proceeding.
assignSystemTables(status);
enableServerShutdownHandler();
status.setStatus("Submitting log splitting work for previously failed region servers");
// Master has recovered hbase:meta region server and we put
// other failed region servers in a queue to be handled later by SSH
@ -908,9 +897,10 @@ MasterServices, Server {
this.clusterStatusChore = getAndStartClusterStatusChore(this);
this.balancerChore = getAndStartBalancerChore(this);
this.catalogJanitorChore = new CatalogJanitor(this, this);
this.namespaceJanitorChore = new NamespaceJanitor(this);
startCatalogJanitorChore();
startNamespaceJanitorChore();
status.setStatus("Starting namespace manager");
initNamespace();
}
if (this.cpHost != null) {
@ -973,17 +963,6 @@ MasterServices, Server {
return new ServerManager(master, services);
}
/**
* If ServerShutdownHandler is disabled, we enable it and expire those dead
* but not expired servers.
*/
private void enableServerShutdownHandler() {
if (!serverShutdownHandlerEnabled) {
serverShutdownHandlerEnabled = true;
this.serverManager.processQueuedDeadServers();
}
}
/**
* Check <code>hbase:meta</code> is assigned. If not, assign it.
* @param status MonitoredTask
@ -996,47 +975,62 @@ MasterServices, Server {
// Work on meta region
int assigned = 0;
long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
boolean beingExpired = false;
status.setStatus("Assigning hbase:meta region");
ServerName logReplayFailedMetaServer = null;
assignmentManager.getRegionStates().createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
boolean rit = this.assignmentManager
.processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
if (!rit && !metaRegionLocation) {
ServerName currentMetaServer = this.catalogTracker.getMetaLocation();
if (currentMetaServer != null) {
beingExpired = expireIfOnline(currentMetaServer);
}
if (beingExpired) {
splitMetaLogBeforeAssignment(currentMetaServer);
}
assignmentManager.assignMeta();
// Make sure a hbase:meta location is set.
enableSSHandWaitForMeta();
if (!metaRegionLocation) {
// Meta location is not verified. It should be in transition, or offline.
// We will wait for it to be assigned in enableSSHandWaitForMeta below.
assigned++;
if (beingExpired && this.distributedLogReplay) {
// In Replay WAL Mode, we need the new hbase:meta server online
this.fileSystemManager.splitMetaLog(currentMetaServer);
if (!rit) {
// Assign meta since not already in transition
ServerName currentMetaServer = this.catalogTracker.getMetaLocation();
if (currentMetaServer != null) {
if (expireIfOnline(currentMetaServer)) {
splitMetaLogBeforeAssignment(currentMetaServer);
if (this.distributedLogReplay) {
logReplayFailedMetaServer = currentMetaServer;
}
}
}
assignmentManager.assignMeta();
}
} else if (rit && !metaRegionLocation) {
// Make sure a hbase:meta location is set.
enableSSHandWaitForMeta();
assigned++;
} else {
// Region already assigned. We didn't assign it. Add to in-memory state.
this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
this.catalogTracker.getMetaLocation());
enableSSHandWaitForMeta();
}
enableMeta(TableName.META_TABLE_NAME);
// Make sure a hbase:meta location is set. We need to enable SSH here since
// if the meta region server is died at this time, we need it to be re-assigned
// by SSH so that system tables can be assigned.
// No need to wait for meta is assigned = 0 when meta is just verified.
enableServerShutdownHandler(assigned != 0);
// logReplayFailedMetaServer is set only if log replay is
// enabled and the current meta server is expired
if (logReplayFailedMetaServer != null) {
// In Replay WAL Mode, we need the new hbase:meta server online
this.fileSystemManager.splitMetaLog(logReplayFailedMetaServer);
}
LOG.info("hbase:meta assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getMetaLocation());
status.setStatus("META assigned.");
}
void initNamespace() throws IOException {
//create namespace manager
tableNamespaceManager = new TableNamespaceManager(this);
tableNamespaceManager.start();
}
private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
if (this.distributedLogReplay) {
// In log replay mode, we mark hbase:meta region as recovering in ZK
@ -1049,107 +1043,24 @@ MasterServices, Server {
}
}
private void splitLogBeforeAssignment(ServerName currentServer,
Set<HRegionInfo> regions) throws IOException {
if (this.distributedLogReplay) {
this.fileSystemManager.prepareLogReplay(currentServer, regions);
} else {
// In recovered.edits mode: create recovered edits file for region server
this.fileSystemManager.splitLog(currentServer);
}
}
void assignSystemTables(MonitoredTask status)
throws InterruptedException, IOException, KeeperException {
// Skip assignment for regions of tables in DISABLING state because during clean cluster startup
// no RS is alive and regions map also doesn't have any information about the regions.
// See HBASE-6281.
Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(zooKeeper);
disabledOrDisablingOrEnabling.addAll(ZKTable.getEnablingTables(zooKeeper));
// Scan hbase:meta for all system regions, skipping any disabled tables
Map<HRegionInfo, ServerName> allRegions =
MetaReader.fullScan(catalogTracker, disabledOrDisablingOrEnabling, true);
for(Iterator<HRegionInfo> iter = allRegions.keySet().iterator();
iter.hasNext();) {
if (!iter.next().getTable().isSystemTable()) {
iter.remove();
}
private void enableServerShutdownHandler(
final boolean waitForMeta) throws IOException, InterruptedException {
// If ServerShutdownHandler is disabled, we enable it and expire those dead
// but not expired servers. This is required so that if meta is assigning to
// a server which dies after assignMeta starts assignment,
// SSH can re-assign it. Otherwise, we will be
// stuck here waiting forever if waitForMeta is specified.
if (!serverShutdownHandlerEnabled) {
serverShutdownHandlerEnabled = true;
this.serverManager.processQueuedDeadServers();
}
boolean beingExpired = false;
status.setStatus("Assigning System Regions");
for (Map.Entry<HRegionInfo, ServerName> entry: allRegions.entrySet()) {
HRegionInfo regionInfo = entry.getKey();
ServerName currServer = entry.getValue();
assignmentManager.getRegionStates().createRegionState(regionInfo);
boolean rit = this.assignmentManager
.processRegionInTransitionAndBlockUntilAssigned(regionInfo);
boolean regionLocation = false;
if (currServer != null) {
regionLocation = verifyRegionLocation(currServer, regionInfo);
}
if (!rit && !regionLocation) {
beingExpired = expireIfOnline(currServer);
if (beingExpired) {
splitLogBeforeAssignment(currServer, Sets.newHashSet(regionInfo));
}
assignmentManager.assign(regionInfo, true);
// Make sure a region location is set.
this.assignmentManager.waitForAssignment(regionInfo);
if (beingExpired && this.distributedLogReplay) {
// In Replay WAL Mode, we need the new region server online
this.fileSystemManager.splitLog(currServer);
}
} else if (rit && !regionLocation) {
if (!waitVerifiedRegionLocation(regionInfo)) return;
} else {
// Region already assigned. We didn't assign it. Add to in-memory state.
this.assignmentManager.regionOnline(regionInfo, currServer);
}
if (!this.assignmentManager.getZKTable().isEnabledTable(regionInfo.getTable())) {
this.assignmentManager.setEnabledTable(regionInfo.getTable());
}
LOG.info("System region " + regionInfo.getRegionNameAsString() + " assigned, rit=" + rit +
", location=" + catalogTracker.getMetaLocation());
if (waitForMeta) {
this.catalogTracker.waitForMeta();
// Above check waits for general meta availability but this does not
// guarantee that the transition has completed
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
}
status.setStatus("System Regions assigned.");
initNamespace();
}
private void enableSSHandWaitForMeta() throws IOException, InterruptedException {
enableServerShutdownHandler();
this.catalogTracker.waitForMeta();
// Above check waits for general meta availability but this does not
// guarantee that the transition has completed
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
}
private boolean waitVerifiedRegionLocation(HRegionInfo regionInfo) throws IOException {
while (!this.stopped) {
Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(catalogTracker,
regionInfo.getRegionName());
if (verifyRegionLocation(p.getSecond(), p.getFirst())) break;
}
// We got here because we came of above loop.
return !this.stopped;
}
private boolean verifyRegionLocation(ServerName currServer, HRegionInfo regionInfo) {
try {
return
ProtobufUtil.getRegionInfo(HConnectionManager.getConnection(conf)
.getAdmin(currServer),
regionInfo.getRegionName()) != null;
} catch (IOException e) {
LOG.info("Failed verifying location=" + currServer + ", exception=" + e);
}
return false;
}
private void enableMeta(TableName metaTableName) {
@ -1174,12 +1085,6 @@ MasterServices, Server {
return true;
}
void initNamespace() throws IOException {
//create namespace manager
tableNamespaceManager = new TableNamespaceManager(this);
tableNamespaceManager.start();
}
/**
* This function returns a set of region server names under hbase:meta recovering region ZK node
* @return Set of meta server names which were recorded in ZK

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import com.google.common.collect.Sets;
@ -63,53 +64,108 @@ public class TableNamespaceManager {
private Configuration conf;
private MasterServices masterServices;
private HTable table;
private HTable nsTable;
private ZKNamespaceManager zkNamespaceManager;
private boolean initialized;
public TableNamespaceManager(MasterServices masterServices) throws IOException {
static final String NS_INIT_TIMEOUT = "hbase.master.namespace.init.timeout";
static final int DEFAULT_NS_INIT_TIMEOUT = 60000;
public TableNamespaceManager(MasterServices masterServices) {
this.masterServices = masterServices;
this.conf = masterServices.getConfiguration();
}
public void start() throws IOException {
TableName tableName = TableName.NAMESPACE_TABLE_NAME;
if (!MetaReader.tableExists(masterServices.getCatalogTracker(),
tableName)) {
LOG.info("Namespace table not found. Creating...");
createNamespaceTable(masterServices);
}
try {
if (!MetaReader.tableExists(masterServices.getCatalogTracker(),
tableName)) {
LOG.info("Namespace table not found. Creating...");
createNamespaceTable(masterServices);
// Wait for the namespace table to be assigned.
// If timed out, we will move ahead without initializing it.
// So that it should be initialized later on lazily.
long startTime = EnvironmentEdgeManager.currentTimeMillis();
int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
while(masterServices.getAssignmentManager()
.getRegionStates().getRegionsOfTable(tableName).isEmpty()) {
if (EnvironmentEdgeManager.currentTimeMillis() - startTime + 100 > timeout) {
LOG.warn("Timedout waiting for namespace table to be assigned.");
return;
}
Thread.sleep(100);
}
} catch (InterruptedException e) {
throw new IOException("Wait for namespace table assignment interrupted", e);
}
table = new HTable(conf, tableName);
zkNamespaceManager = new ZKNamespaceManager(masterServices.getZooKeeper());
zkNamespaceManager.start();
if (get(NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
create(NamespaceDescriptor.DEFAULT_NAMESPACE);
}
if (get(NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
create(NamespaceDescriptor.SYSTEM_NAMESPACE);
}
// initialize namespace table
getNamespaceTable();
}
ResultScanner scanner = table.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
try {
for(Result result : scanner) {
byte[] val = CellUtil.cloneValue(result.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES));
NamespaceDescriptor ns =
ProtobufUtil.toNamespaceDescriptor(
HBaseProtos.NamespaceDescriptor.parseFrom(val));
zkNamespaceManager.update(ns);
@SuppressWarnings("deprecation")
private synchronized HTable getNamespaceTable() throws IOException {
if (!initialized) {
try {
nsTable = new HTable(conf, TableName.NAMESPACE_TABLE_NAME);
zkNamespaceManager = new ZKNamespaceManager(masterServices.getZooKeeper());
zkNamespaceManager.start();
if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
create(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE);
}
if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
create(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE);
}
ResultScanner scanner = nsTable.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
try {
for(Result result : scanner) {
byte[] val = CellUtil.cloneValue(result.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES));
NamespaceDescriptor ns =
ProtobufUtil.toNamespaceDescriptor(
HBaseProtos.NamespaceDescriptor.parseFrom(val));
zkNamespaceManager.update(ns);
}
} finally {
scanner.close();
}
initialized = true;
} catch (IOException ie) {
LOG.warn("Caught exception in initializing namespace table manager", ie);
if (nsTable != null) {
nsTable.close();
}
throw ie;
}
} finally {
scanner.close();
} else if (nsTable.getConnection().isClosed()) {
nsTable = new HTable(conf, TableName.NAMESPACE_TABLE_NAME);
}
return nsTable;
}
public synchronized NamespaceDescriptor get(String name) throws IOException {
return get(getNamespaceTable(), name);
}
public synchronized void create(NamespaceDescriptor ns) throws IOException {
create(getNamespaceTable(), ns);
}
public synchronized void update(NamespaceDescriptor ns) throws IOException {
HTable table = getNamespaceTable();
if (get(table, ns.getName()) == null) {
throw new ConstraintException("Namespace "+ns.getName()+" does not exist");
}
upsert(table, ns);
}
private NamespaceDescriptor get(HTable table, String name) throws IOException {
Result res = table.get(new Get(Bytes.toBytes(name)));
if (res.isEmpty()) {
return null;
@ -121,24 +177,17 @@ public class TableNamespaceManager {
HBaseProtos.NamespaceDescriptor.parseFrom(val));
}
public synchronized void create(NamespaceDescriptor ns) throws IOException {
if (get(ns.getName()) != null) {
private void create(HTable table, NamespaceDescriptor ns) throws IOException {
if (get(table, ns.getName()) != null) {
throw new ConstraintException("Namespace "+ns.getName()+" already exists");
}
FileSystem fs = masterServices.getMasterFileSystem().getFileSystem();
fs.mkdirs(FSUtils.getNamespaceDir(
masterServices.getMasterFileSystem().getRootDir(), ns.getName()));
upsert(ns);
upsert(table, ns);
}
public synchronized void update(NamespaceDescriptor ns) throws IOException {
if (get(ns.getName()) == null) {
throw new ConstraintException("Namespace "+ns.getName()+" does not exist");
}
upsert(ns);
}
private void upsert(NamespaceDescriptor ns) throws IOException {
private void upsert(HTable table, NamespaceDescriptor ns) throws IOException {
Put p = new Put(Bytes.toBytes(ns.getName()));
p.add(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES,
@ -168,7 +217,7 @@ public class TableNamespaceManager {
"Namespace "+name+" has "+tableCount+" tables");
}
Delete d = new Delete(Bytes.toBytes(name));
table.delete(d);
getNamespaceTable().delete(d);
//don't abort if cleanup isn't complete
//it will be replaced on new namespace creation
zkNamespaceManager.remove(name);
@ -189,7 +238,7 @@ public class TableNamespaceManager {
public synchronized NavigableSet<NamespaceDescriptor> list() throws IOException {
NavigableSet<NamespaceDescriptor> ret =
Sets.newTreeSet(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR);
ResultScanner scanner = table.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
ResultScanner scanner = getNamespaceTable().getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
try {
for(Result r : scanner) {
byte[] val = CellUtil.cloneValue(r.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
@ -203,7 +252,7 @@ public class TableNamespaceManager {
return ret;
}
private void createNamespaceTable(MasterServices masterServices) throws IOException, InterruptedException {
private void createNamespaceTable(MasterServices masterServices) throws IOException {
HRegionInfo newRegions[] = new HRegionInfo[]{
new HRegionInfo(HTableDescriptor.NAMESPACE_TABLEDESC.getTableName(), null, null)};
@ -216,16 +265,5 @@ public class TableNamespaceManager {
masterServices.getConfiguration(),
newRegions,
masterServices).prepare());
//wait for region to be online
int tries = conf.getInt("hbase.master.namespace.init.timeout", 600);
while(masterServices.getAssignmentManager()
.getRegionStates().getRegionServerOfRegion(newRegions[0]) == null &&
tries > 0) {
Thread.sleep(100);
tries--;
}
if (tries <= 0) {
throw new IOException("Failed to create namespace table.");
}
}
}

View File

@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.migration.NamespaceUpgrade;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -280,7 +280,7 @@ public class TestNamespace {
admin.deleteTable(desc.getTableName());
}
@Test
@Ignore @Test
public void testNamespaceJanitor() throws Exception {
FileSystem fs = TEST_UTIL.getTestFileSystem();

View File

@ -211,8 +211,7 @@ public class TestMasterNoCluster {
}
@Override
void assignSystemTables(MonitoredTask status)
throws IOException, InterruptedException, KeeperException {
void initNamespace() {
}
};
master.start();
@ -302,8 +301,7 @@ public class TestMasterNoCluster {
}
@Override
void assignSystemTables(MonitoredTask status)
throws IOException, InterruptedException, KeeperException {
void initNamespace() {
}
};
master.start();
@ -366,7 +364,6 @@ public class TestMasterNoCluster {
HMaster master = new HMaster(conf) {
@Override
void assignMeta(MonitoredTask status) {
return;
}
@Override
@ -400,8 +397,7 @@ public class TestMasterNoCluster {
}
@Override
void assignSystemTables(MonitoredTask status)
throws IOException, InterruptedException, KeeperException {
void initNamespace() {
}
};
master.start();