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:
parent
1f969f6155
commit
704a00b544
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
Loading…
Reference in New Issue