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

View File

@ -29,7 +29,6 @@ import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -43,7 +42,6 @@ import java.util.concurrent.atomic.AtomicReference;
import javax.management.ObjectName; import javax.management.ObjectName;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; 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.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorType; import org.apache.hadoop.hbase.executor.ExecutorType;
import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; 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;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.RequestContext; import org.apache.hadoop.hbase.ipc.RequestContext;
import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.ipc.ServerRpcController; 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.BalancerChore;
import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore; import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory; 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.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.RegionServerTracker; import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId; 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.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -875,12 +870,6 @@ MasterServices, Server {
this.fileSystemManager.splitMetaLog(previouslyFailedMetaRSs); 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"); status.setStatus("Submitting log splitting work for previously failed region servers");
// Master has recovered hbase:meta region server and we put // Master has recovered hbase:meta region server and we put
// other failed region servers in a queue to be handled later by SSH // other failed region servers in a queue to be handled later by SSH
@ -908,9 +897,10 @@ MasterServices, Server {
this.clusterStatusChore = getAndStartClusterStatusChore(this); this.clusterStatusChore = getAndStartClusterStatusChore(this);
this.balancerChore = getAndStartBalancerChore(this); this.balancerChore = getAndStartBalancerChore(this);
this.catalogJanitorChore = new CatalogJanitor(this, this); this.catalogJanitorChore = new CatalogJanitor(this, this);
this.namespaceJanitorChore = new NamespaceJanitor(this);
startCatalogJanitorChore(); startCatalogJanitorChore();
startNamespaceJanitorChore();
status.setStatus("Starting namespace manager");
initNamespace();
} }
if (this.cpHost != null) { if (this.cpHost != null) {
@ -973,17 +963,6 @@ MasterServices, Server {
return new ServerManager(master, services); 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. * Check <code>hbase:meta</code> is assigned. If not, assign it.
* @param status MonitoredTask * @param status MonitoredTask
@ -996,47 +975,62 @@ MasterServices, Server {
// Work on meta region // Work on meta region
int assigned = 0; int assigned = 0;
long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000); long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
boolean beingExpired = false;
status.setStatus("Assigning hbase:meta region"); status.setStatus("Assigning hbase:meta region");
ServerName logReplayFailedMetaServer = null;
assignmentManager.getRegionStates().createRegionState(HRegionInfo.FIRST_META_REGIONINFO); assignmentManager.getRegionStates().createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
boolean rit = this.assignmentManager boolean rit = this.assignmentManager
.processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO); .processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout); boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
if (!rit && !metaRegionLocation) { 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 (!rit) {
// Assign meta since not already in transition
ServerName currentMetaServer = this.catalogTracker.getMetaLocation(); ServerName currentMetaServer = this.catalogTracker.getMetaLocation();
if (currentMetaServer != null) { if (currentMetaServer != null) {
beingExpired = expireIfOnline(currentMetaServer); if (expireIfOnline(currentMetaServer)) {
}
if (beingExpired) {
splitMetaLogBeforeAssignment(currentMetaServer); splitMetaLogBeforeAssignment(currentMetaServer);
if (this.distributedLogReplay) {
logReplayFailedMetaServer = currentMetaServer;
}
}
} }
assignmentManager.assignMeta(); assignmentManager.assignMeta();
// Make sure a hbase:meta location is set.
enableSSHandWaitForMeta();
assigned++;
if (beingExpired && this.distributedLogReplay) {
// In Replay WAL Mode, we need the new hbase:meta server online
this.fileSystemManager.splitMetaLog(currentMetaServer);
} }
} else if (rit && !metaRegionLocation) {
// Make sure a hbase:meta location is set.
enableSSHandWaitForMeta();
assigned++;
} else { } else {
// Region already assigned. We didn't assign it. Add to in-memory state. // Region already assigned. We didn't assign it. Add to in-memory state.
this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO, this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
this.catalogTracker.getMetaLocation()); this.catalogTracker.getMetaLocation());
enableSSHandWaitForMeta();
} }
enableMeta(TableName.META_TABLE_NAME); 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 + LOG.info("hbase:meta assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getMetaLocation()); ", location=" + catalogTracker.getMetaLocation());
status.setStatus("META assigned."); status.setStatus("META assigned.");
} }
void initNamespace() throws IOException {
//create namespace manager
tableNamespaceManager = new TableNamespaceManager(this);
tableNamespaceManager.start();
}
private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException { private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
if (this.distributedLogReplay) { if (this.distributedLogReplay) {
// In log replay mode, we mark hbase:meta region as recovering in ZK // In log replay mode, we mark hbase:meta region as recovering in ZK
@ -1049,107 +1043,24 @@ MasterServices, Server {
} }
} }
private void splitLogBeforeAssignment(ServerName currentServer, private void enableServerShutdownHandler(
Set<HRegionInfo> regions) throws IOException { final boolean waitForMeta) throws IOException, InterruptedException {
if (this.distributedLogReplay) { // If ServerShutdownHandler is disabled, we enable it and expire those dead
this.fileSystemManager.prepareLogReplay(currentServer, regions); // but not expired servers. This is required so that if meta is assigning to
} else { // a server which dies after assignMeta starts assignment,
// In recovered.edits mode: create recovered edits file for region server // SSH can re-assign it. Otherwise, we will be
this.fileSystemManager.splitLog(currentServer); // stuck here waiting forever if waitForMeta is specified.
} if (!serverShutdownHandlerEnabled) {
serverShutdownHandlerEnabled = true;
this.serverManager.processQueuedDeadServers();
} }
void assignSystemTables(MonitoredTask status) if (waitForMeta) {
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();
}
}
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());
}
status.setStatus("System Regions assigned.");
initNamespace();
}
private void enableSSHandWaitForMeta() throws IOException, InterruptedException {
enableServerShutdownHandler();
this.catalogTracker.waitForMeta(); this.catalogTracker.waitForMeta();
// Above check waits for general meta availability but this does not // Above check waits for general meta availability but this does not
// guarantee that the transition has completed // guarantee that the transition has completed
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO); 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) { private void enableMeta(TableName metaTableName) {
@ -1174,12 +1085,6 @@ MasterServices, Server {
return true; 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 * 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 * @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.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
@ -63,37 +64,64 @@ public class TableNamespaceManager {
private Configuration conf; private Configuration conf;
private MasterServices masterServices; private MasterServices masterServices;
private HTable table; private HTable nsTable;
private ZKNamespaceManager zkNamespaceManager; 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.masterServices = masterServices;
this.conf = masterServices.getConfiguration(); this.conf = masterServices.getConfiguration();
} }
public void start() throws IOException { public void start() throws IOException {
TableName tableName = TableName.NAMESPACE_TABLE_NAME; TableName tableName = TableName.NAMESPACE_TABLE_NAME;
try {
if (!MetaReader.tableExists(masterServices.getCatalogTracker(), if (!MetaReader.tableExists(masterServices.getCatalogTracker(),
tableName)) { tableName)) {
LOG.info("Namespace table not found. Creating..."); LOG.info("Namespace table not found. Creating...");
createNamespaceTable(masterServices); createNamespaceTable(masterServices);
} }
try {
// 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) { } catch (InterruptedException e) {
throw new IOException("Wait for namespace table assignment interrupted", e); throw new IOException("Wait for namespace table assignment interrupted", e);
} }
table = new HTable(conf, tableName);
// initialize namespace table
getNamespaceTable();
}
@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 = new ZKNamespaceManager(masterServices.getZooKeeper());
zkNamespaceManager.start(); zkNamespaceManager.start();
if (get(NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) { if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
create(NamespaceDescriptor.DEFAULT_NAMESPACE); create(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE);
} }
if (get(NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) { if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
create(NamespaceDescriptor.SYSTEM_NAMESPACE); create(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE);
} }
ResultScanner scanner = table.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES); ResultScanner scanner = nsTable.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
try { try {
for(Result result : scanner) { for(Result result : scanner) {
byte[] val = CellUtil.cloneValue(result.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES, byte[] val = CellUtil.cloneValue(result.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
@ -106,10 +134,38 @@ public class TableNamespaceManager {
} finally { } finally {
scanner.close(); scanner.close();
} }
initialized = true;
} catch (IOException ie) {
LOG.warn("Caught exception in initializing namespace table manager", ie);
if (nsTable != null) {
nsTable.close();
}
throw ie;
}
} else if (nsTable.getConnection().isClosed()) {
nsTable = new HTable(conf, TableName.NAMESPACE_TABLE_NAME);
}
return nsTable;
} }
public synchronized NamespaceDescriptor get(String name) throws IOException { 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))); Result res = table.get(new Get(Bytes.toBytes(name)));
if (res.isEmpty()) { if (res.isEmpty()) {
return null; return null;
@ -121,24 +177,17 @@ public class TableNamespaceManager {
HBaseProtos.NamespaceDescriptor.parseFrom(val)); HBaseProtos.NamespaceDescriptor.parseFrom(val));
} }
public synchronized void create(NamespaceDescriptor ns) throws IOException { private void create(HTable table, NamespaceDescriptor ns) throws IOException {
if (get(ns.getName()) != null) { if (get(table, ns.getName()) != null) {
throw new ConstraintException("Namespace "+ns.getName()+" already exists"); throw new ConstraintException("Namespace "+ns.getName()+" already exists");
} }
FileSystem fs = masterServices.getMasterFileSystem().getFileSystem(); FileSystem fs = masterServices.getMasterFileSystem().getFileSystem();
fs.mkdirs(FSUtils.getNamespaceDir( fs.mkdirs(FSUtils.getNamespaceDir(
masterServices.getMasterFileSystem().getRootDir(), ns.getName())); masterServices.getMasterFileSystem().getRootDir(), ns.getName()));
upsert(ns); upsert(table, ns);
} }
public synchronized void update(NamespaceDescriptor ns) throws IOException { private void upsert(HTable table, 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 {
Put p = new Put(Bytes.toBytes(ns.getName())); Put p = new Put(Bytes.toBytes(ns.getName()));
p.add(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES, p.add(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES, HTableDescriptor.NAMESPACE_COL_DESC_BYTES,
@ -168,7 +217,7 @@ public class TableNamespaceManager {
"Namespace "+name+" has "+tableCount+" tables"); "Namespace "+name+" has "+tableCount+" tables");
} }
Delete d = new Delete(Bytes.toBytes(name)); Delete d = new Delete(Bytes.toBytes(name));
table.delete(d); getNamespaceTable().delete(d);
//don't abort if cleanup isn't complete //don't abort if cleanup isn't complete
//it will be replaced on new namespace creation //it will be replaced on new namespace creation
zkNamespaceManager.remove(name); zkNamespaceManager.remove(name);
@ -189,7 +238,7 @@ public class TableNamespaceManager {
public synchronized NavigableSet<NamespaceDescriptor> list() throws IOException { public synchronized NavigableSet<NamespaceDescriptor> list() throws IOException {
NavigableSet<NamespaceDescriptor> ret = NavigableSet<NamespaceDescriptor> ret =
Sets.newTreeSet(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR); 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 { try {
for(Result r : scanner) { for(Result r : scanner) {
byte[] val = CellUtil.cloneValue(r.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES, byte[] val = CellUtil.cloneValue(r.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
@ -203,7 +252,7 @@ public class TableNamespaceManager {
return ret; return ret;
} }
private void createNamespaceTable(MasterServices masterServices) throws IOException, InterruptedException { private void createNamespaceTable(MasterServices masterServices) throws IOException {
HRegionInfo newRegions[] = new HRegionInfo[]{ HRegionInfo newRegions[] = new HRegionInfo[]{
new HRegionInfo(HTableDescriptor.NAMESPACE_TABLEDESC.getTableName(), null, null)}; new HRegionInfo(HTableDescriptor.NAMESPACE_TABLEDESC.getTableName(), null, null)};
@ -216,16 +265,5 @@ public class TableNamespaceManager {
masterServices.getConfiguration(), masterServices.getConfiguration(),
newRegions, newRegions,
masterServices).prepare()); 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.client.Put;
import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.HMaster; 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.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.ZKUtil; 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.AfterClass;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
@ -280,7 +280,7 @@ public class TestNamespace {
admin.deleteTable(desc.getTableName()); admin.deleteTable(desc.getTableName());
} }
@Test @Ignore @Test
public void testNamespaceJanitor() throws Exception { public void testNamespaceJanitor() throws Exception {
FileSystem fs = TEST_UTIL.getTestFileSystem(); FileSystem fs = TEST_UTIL.getTestFileSystem();

View File

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